You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by li...@apache.org on 2018/09/12 18:25:29 UTC
[1/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Repository: spark
Updated Branches:
refs/heads/branch-2.4 4c1428fa2 -> 15d2e9d7d
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala
index aeef4c8..52b833a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala
@@ -17,74 +17,73 @@
package org.apache.spark.sql.streaming.sources
+import java.util.Optional
+
import org.apache.spark.sql.{DataFrame, SQLContext}
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, StreamingQueryWrapper}
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
-import org.apache.spark.sql.sources.v2._
-import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, ScanConfig, ScanConfigBuilder}
-import org.apache.spark.sql.sources.v2.reader.streaming._
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.reader.InputPartition
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader, Offset, PartitionOffset}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.{OutputMode, StreamTest, Trigger}
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
-case class FakeReadSupport() extends MicroBatchReadSupport with ContinuousReadSupport {
- override def deserializeOffset(json: String): Offset = RateStreamOffset(Map())
- override def commit(end: Offset): Unit = {}
- override def stop(): Unit = {}
- override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map())
- override def fullSchema(): StructType = StructType(Seq())
- override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = null
- override def initialOffset(): Offset = RateStreamOffset(Map())
- override def latestOffset(): Offset = RateStreamOffset(Map())
- override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = null
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- throw new IllegalStateException("fake source - cannot actually read")
- }
- override def createContinuousReaderFactory(
- config: ScanConfig): ContinuousPartitionReaderFactory = {
- throw new IllegalStateException("fake source - cannot actually read")
- }
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
+case class FakeReader() extends MicroBatchReader with ContinuousReader {
+ def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = {}
+ def getStartOffset: Offset = RateStreamOffset(Map())
+ def getEndOffset: Offset = RateStreamOffset(Map())
+ def deserializeOffset(json: String): Offset = RateStreamOffset(Map())
+ def commit(end: Offset): Unit = {}
+ def readSchema(): StructType = StructType(Seq())
+ def stop(): Unit = {}
+ def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map())
+ def setStartOffset(start: Optional[Offset]): Unit = {}
+
+ def planInputPartitions(): java.util.ArrayList[InputPartition[InternalRow]] = {
throw new IllegalStateException("fake source - cannot actually read")
}
}
-trait FakeMicroBatchReadSupportProvider extends MicroBatchReadSupportProvider {
- override def createMicroBatchReadSupport(
+trait FakeMicroBatchReadSupport extends MicroBatchReadSupport {
+ override def createMicroBatchReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): MicroBatchReadSupport = FakeReadSupport()
+ options: DataSourceOptions): MicroBatchReader = FakeReader()
}
-trait FakeContinuousReadSupportProvider extends ContinuousReadSupportProvider {
- override def createContinuousReadSupport(
+trait FakeContinuousReadSupport extends ContinuousReadSupport {
+ override def createContinuousReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): ContinuousReadSupport = FakeReadSupport()
+ options: DataSourceOptions): ContinuousReader = FakeReader()
}
-trait FakeStreamingWriteSupportProvider extends StreamingWriteSupportProvider {
- override def createStreamingWriteSupport(
+trait FakeStreamWriteSupport extends StreamWriteSupport {
+ override def createStreamWriter(
queryId: String,
schema: StructType,
mode: OutputMode,
- options: DataSourceOptions): StreamingWriteSupport = {
+ options: DataSourceOptions): StreamWriter = {
throw new IllegalStateException("fake sink - cannot actually write")
}
}
-class FakeReadMicroBatchOnly extends DataSourceRegister with FakeMicroBatchReadSupportProvider {
+class FakeReadMicroBatchOnly extends DataSourceRegister with FakeMicroBatchReadSupport {
override def shortName(): String = "fake-read-microbatch-only"
}
-class FakeReadContinuousOnly extends DataSourceRegister with FakeContinuousReadSupportProvider {
+class FakeReadContinuousOnly extends DataSourceRegister with FakeContinuousReadSupport {
override def shortName(): String = "fake-read-continuous-only"
}
class FakeReadBothModes extends DataSourceRegister
- with FakeMicroBatchReadSupportProvider with FakeContinuousReadSupportProvider {
+ with FakeMicroBatchReadSupport with FakeContinuousReadSupport {
override def shortName(): String = "fake-read-microbatch-continuous"
}
@@ -92,7 +91,7 @@ class FakeReadNeitherMode extends DataSourceRegister {
override def shortName(): String = "fake-read-neither-mode"
}
-class FakeWriteSupportProvider extends DataSourceRegister with FakeStreamingWriteSupportProvider {
+class FakeWrite extends DataSourceRegister with FakeStreamWriteSupport {
override def shortName(): String = "fake-write-microbatch-continuous"
}
@@ -107,8 +106,8 @@ class FakeSink extends Sink {
override def addBatch(batchId: Long, data: DataFrame): Unit = {}
}
-class FakeWriteSupportProviderV1Fallback extends DataSourceRegister
- with FakeStreamingWriteSupportProvider with StreamSinkProvider {
+class FakeWriteV1Fallback extends DataSourceRegister
+ with FakeStreamWriteSupport with StreamSinkProvider {
override def createSink(
sqlContext: SQLContext,
@@ -191,11 +190,11 @@ class StreamingDataSourceV2Suite extends StreamTest {
val v2Query = testPositiveCase(
"fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once())
assert(v2Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink
- .isInstanceOf[FakeWriteSupportProviderV1Fallback])
+ .isInstanceOf[FakeWriteV1Fallback])
// Ensure we create a V1 sink with the config. Note the config is a comma separated
// list, including other fake entries.
- val fullSinkName = classOf[FakeWriteSupportProviderV1Fallback].getName
+ val fullSinkName = "org.apache.spark.sql.streaming.sources.FakeWriteV1Fallback"
withSQLConf(SQLConf.DISABLED_V2_STREAMING_WRITERS.key -> s"a,b,c,test,$fullSinkName,d,e") {
val v1Query = testPositiveCase(
"fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once())
@@ -219,37 +218,35 @@ class StreamingDataSourceV2Suite extends StreamTest {
val writeSource = DataSource.lookupDataSource(write, spark.sqlContext.conf).newInstance()
(readSource, writeSource, trigger) match {
// Valid microbatch queries.
- case (_: MicroBatchReadSupportProvider, _: StreamingWriteSupportProvider, t)
+ case (_: MicroBatchReadSupport, _: StreamWriteSupport, t)
if !t.isInstanceOf[ContinuousTrigger] =>
testPositiveCase(read, write, trigger)
// Valid continuous queries.
- case (_: ContinuousReadSupportProvider, _: StreamingWriteSupportProvider,
- _: ContinuousTrigger) =>
+ case (_: ContinuousReadSupport, _: StreamWriteSupport, _: ContinuousTrigger) =>
testPositiveCase(read, write, trigger)
// Invalid - can't read at all
case (r, _, _)
- if !r.isInstanceOf[MicroBatchReadSupportProvider]
- && !r.isInstanceOf[ContinuousReadSupportProvider] =>
+ if !r.isInstanceOf[MicroBatchReadSupport]
+ && !r.isInstanceOf[ContinuousReadSupport] =>
testNegativeCase(read, write, trigger,
s"Data source $read does not support streamed reading")
// Invalid - can't write
- case (_, w, _) if !w.isInstanceOf[StreamingWriteSupportProvider] =>
+ case (_, w, _) if !w.isInstanceOf[StreamWriteSupport] =>
testNegativeCase(read, write, trigger,
s"Data source $write does not support streamed writing")
// Invalid - trigger is continuous but reader is not
- case (r, _: StreamingWriteSupportProvider, _: ContinuousTrigger)
- if !r.isInstanceOf[ContinuousReadSupportProvider] =>
+ case (r, _: StreamWriteSupport, _: ContinuousTrigger)
+ if !r.isInstanceOf[ContinuousReadSupport] =>
testNegativeCase(read, write, trigger,
s"Data source $read does not support continuous processing")
// Invalid - trigger is microbatch but reader is not
case (r, _, t)
- if !r.isInstanceOf[MicroBatchReadSupportProvider] &&
- !t.isInstanceOf[ContinuousTrigger] =>
+ if !r.isInstanceOf[MicroBatchReadSupport] && !t.isInstanceOf[ContinuousTrigger] =>
testPostCreationNegativeCase(read, write, trigger,
s"Data source $read does not support microbatch processing")
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[6/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java
deleted file mode 100644
index f403dc6..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils;
-import org.apache.spark.sql.sources.v2.reader.BatchReadSupport;
-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 batch processing.
- *
- * This interface is used to create {@link BatchReadSupport} instances when end users run
- * {@code SparkSession.read.format(...).option(...).load()}.
- */
-@InterfaceStability.Evolving
-public interface BatchReadSupportProvider extends DataSourceV2 {
-
- /**
- * Creates a {@link BatchReadSupport} instance to load the data from this data source with a user
- * specified schema, which is called by Spark at the beginning of each batch query.
- *
- * Spark will call this method at the beginning of each batch query to create a
- * {@link BatchReadSupport} instance.
- *
- * By default this method throws {@link UnsupportedOperationException}, implementations should
- * override this method to handle user specified schema.
- *
- * @param schema the user specified schema.
- * @param options the options for the returned data source reader, which is an immutable
- * case-insensitive string-to-string map.
- */
- default BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) {
- return DataSourceV2Utils.failForUserSpecifiedSchema(this);
- }
-
- /**
- * Creates a {@link BatchReadSupport} instance to scan the data from this data source, which is
- * called by Spark at the beginning of each batch query.
- *
- * @param options the options for the returned data source reader, which is an immutable
- * case-insensitive string-to-string map.
- */
- BatchReadSupport createBatchReadSupport(DataSourceOptions options);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java
deleted file mode 100644
index bd10c33..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2;
-
-import java.util.Optional;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.SaveMode;
-import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport;
-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 batch processing.
- *
- * This interface is used to create {@link BatchWriteSupport} instances when end users run
- * {@code Dataset.write.format(...).option(...).save()}.
- */
-@InterfaceStability.Evolving
-public interface BatchWriteSupportProvider extends DataSourceV2 {
-
- /**
- * Creates an optional {@link BatchWriteSupport} instance to save the data to this data source,
- * which is called by Spark at the beginning of each batch query.
- *
- * Data sources can return None if there is no writing needed to be done according to the save
- * mode.
- *
- * @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 BatchWriteSupport} can use this id to distinguish itself from others.
- * @param schema the schema of the data to be written.
- * @param mode the save mode which determines what to do when the data are already in this data
- * source, please refer to {@link SaveMode} for more details.
- * @param options the options for the returned data source writer, which is an immutable
- * case-insensitive string-to-string map.
- * @return a write support to write data to this data source.
- */
- Optional<BatchWriteSupport> createBatchWriteSupport(
- String queryId,
- StructType schema,
- SaveMode mode,
- DataSourceOptions options);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java
----------------------------------------------------------------------
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 0000000..7df5a45
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader;
+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.
+ */
+@InterfaceStability.Evolving
+public interface ContinuousReadSupport extends DataSourceV2 {
+ /**
+ * 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 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.
+ */
+ ContinuousReader createContinuousReader(
+ Optional<StructType> schema,
+ String checkpointLocation,
+ DataSourceOptions options);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java
deleted file mode 100644
index 824c290..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils;
-import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport;
-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.
- *
- * This interface is used to create {@link ContinuousReadSupport} instances when end users run
- * {@code SparkSession.readStream.format(...).option(...).load()} with a continuous trigger.
- */
-@InterfaceStability.Evolving
-public interface ContinuousReadSupportProvider extends DataSourceV2 {
-
- /**
- * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data
- * source with a user specified schema, which is called by Spark at the beginning of each
- * continuous streaming query.
- *
- * By default this method throws {@link UnsupportedOperationException}, implementations should
- * override this method to handle user specified schema.
- *
- * @param schema the user provided schema.
- * @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.
- */
- default ContinuousReadSupport createContinuousReadSupport(
- StructType schema,
- String checkpointLocation,
- DataSourceOptions options) {
- return DataSourceV2Utils.failForUserSpecifiedSchema(this);
- }
-
- /**
- * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data
- * source, which is called by Spark at the beginning of each continuous streaming query.
- *
- * @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.
- */
- ContinuousReadSupport createContinuousReadSupport(
- String checkpointLocation,
- DataSourceOptions options);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java
index 6e31e84..6234071 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java
@@ -22,13 +22,9 @@ import org.apache.spark.annotation.InterfaceStability;
/**
* The base interface for data source v2. Implementations must have a public, 0-arg constructor.
*
- * Note that this is an empty interface. Data source implementations must mix in interfaces such as
- * {@link BatchReadSupportProvider} or {@link BatchWriteSupportProvider}, which can provide
- * batch or streaming read/write support instances. Otherwise it's just a dummy data source which
- * is un-readable/writable.
- *
- * If Spark fails to execute any methods in the implementations of this interface (by throwing an
- * exception), the read action will fail and no Spark job will be submitted.
+ * Note that this is an empty interface. Data source implementations should mix-in at least one of
+ * the plug-in interfaces like {@link ReadSupport} and {@link WriteSupport}. Otherwise it's just
+ * a dummy data source which is un-readable/writable.
*/
@InterfaceStability.Evolving
public interface DataSourceV2 {}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java
----------------------------------------------------------------------
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 0000000..7f4a2c9
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.reader.streaming.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 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 planInputPartitions 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
+ * 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.
+ */
+ MicroBatchReader createMicroBatchReader(
+ Optional<StructType> schema,
+ String checkpointLocation,
+ DataSourceOptions options);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java
deleted file mode 100644
index 61c08e7..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils;
-import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport;
-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 micro-batch stream processing.
- *
- * This interface is used to create {@link MicroBatchReadSupport} instances when end users run
- * {@code SparkSession.readStream.format(...).option(...).load()} with a micro-batch trigger.
- */
-@InterfaceStability.Evolving
-public interface MicroBatchReadSupportProvider extends DataSourceV2 {
-
- /**
- * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data
- * source with a user specified schema, which is called by Spark at the beginning of each
- * micro-batch streaming query.
- *
- * By default this method throws {@link UnsupportedOperationException}, implementations should
- * override this method to handle user specified schema.
- *
- * @param schema the user provided schema.
- * @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.
- */
- default MicroBatchReadSupport createMicroBatchReadSupport(
- StructType schema,
- String checkpointLocation,
- DataSourceOptions options) {
- return DataSourceV2Utils.failForUserSpecifiedSchema(this);
- }
-
- /**
- * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data
- * source, which is called by Spark at the beginning of each micro-batch streaming query.
- *
- * @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.
- */
- MicroBatchReadSupport createMicroBatchReadSupport(
- String checkpointLocation,
- DataSourceOptions options);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java
new file mode 100644
index 0000000..80ac08e
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java
@@ -0,0 +1,65 @@
+/*
+ * 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 org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+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 and scan the data from the data source.
+ */
+@InterfaceStability.Evolving
+public interface ReadSupport extends DataSourceV2 {
+
+ /**
+ * Creates a {@link DataSourceReader} to scan the data from this data source.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ *
+ * @param schema the user specified schema.
+ * @param options the options for the returned data source reader, which is an immutable
+ * case-insensitive string-to-string map.
+ *
+ * By default this method throws {@link UnsupportedOperationException}, implementations should
+ * override this method to handle user specified schema.
+ */
+ default DataSourceReader createReader(StructType schema, DataSourceOptions options) {
+ String name;
+ if (this instanceof DataSourceRegister) {
+ name = ((DataSourceRegister) this).shortName();
+ } else {
+ name = this.getClass().getName();
+ }
+ throw new UnsupportedOperationException(name + " does not support user specified schema");
+ }
+
+ /**
+ * Creates a {@link DataSourceReader} to scan the data from this data source.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ *
+ * @param options the options for the returned data source reader, which is an immutable
+ * case-insensitive string-to-string map.
+ */
+ DataSourceReader createReader(DataSourceOptions options);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java
index bbe430e..926c6fd 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java
@@ -28,10 +28,9 @@ import org.apache.spark.annotation.InterfaceStability;
public interface SessionConfigSupport extends DataSourceV2 {
/**
- * Key prefix of the session configs to propagate, which is usually the data source name. Spark
- * will extract all session configs that starts with `spark.datasource.$keyPrefix`, turn
- * `spark.datasource.$keyPrefix.xxx -> yyy` into `xxx -> yyy`, and propagate them to all
- * data source operations in this session.
+ * Key prefix of the session configs to propagate. Spark will extract all session configs that
+ * starts with `spark.datasource.$keyPrefix`, turn `spark.datasource.$keyPrefix.xxx -> yyy`
+ * into `xxx -> yyy`, and propagate them to all data source operations in this session.
*/
String keyPrefix();
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java
new file mode 100644
index 0000000..a77b014
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java
@@ -0,0 +1,52 @@
+/*
+ * 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 org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.execution.streaming.BaseStreamingSink;
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter;
+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 structured streaming.
+ */
+@InterfaceStability.Evolving
+public interface StreamWriteSupport extends DataSourceV2, BaseStreamingSink {
+
+ /**
+ * Creates an optional {@link StreamWriter} 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 queries running at the same time, and the returned
+ * {@link DataSourceWriter} 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 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.
+ */
+ StreamWriter createStreamWriter(
+ String queryId,
+ StructType schema,
+ OutputMode mode,
+ DataSourceOptions options);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java
deleted file mode 100644
index f9ca85d..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.streaming.BaseStreamingSink;
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport;
-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 structured streaming.
- *
- * This interface is used to create {@link StreamingWriteSupport} instances when end users run
- * {@code Dataset.writeStream.format(...).option(...).start()}.
- */
-@InterfaceStability.Evolving
-public interface StreamingWriteSupportProvider extends DataSourceV2, BaseStreamingSink {
-
- /**
- * Creates a {@link StreamingWriteSupport} instance to save the data to this data source, which is
- * called by Spark at the beginning of each streaming query.
- *
- * @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 StreamingWriteSupport} 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 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.
- */
- StreamingWriteSupport createStreamingWriteSupport(
- String queryId,
- StructType schema,
- OutputMode mode,
- DataSourceOptions options);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java
new file mode 100644
index 0000000..048787a
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
+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 to the data source.
+ */
+@InterfaceStability.Evolving
+public interface WriteSupport extends DataSourceV2 {
+
+ /**
+ * Creates an optional {@link DataSourceWriter} to save the data to this data source. Data
+ * sources can return None if there is no writing needed to be done according to the save mode.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ *
+ * @param writeUUID A unique string for the writing job. It's possible that there are many writing
+ * jobs running at the same time, and the returned {@link DataSourceWriter} can
+ * use this job id to distinguish itself from other jobs.
+ * @param schema the schema of the data to be written.
+ * @param mode the save mode which determines what to do when the data are already in this data
+ * source, please refer to {@link SaveMode} for more details.
+ * @param options the options for the returned data source writer, which is an immutable
+ * case-insensitive string-to-string map.
+ * @return a writer to append data to this data source
+ */
+ Optional<DataSourceWriter> createWriter(
+ String writeUUID, StructType schema, SaveMode mode, DataSourceOptions options);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java
deleted file mode 100644
index 452ee86..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import org.apache.spark.annotation.InterfaceStability;
-
-/**
- * An interface that defines how to load the data from data source for batch processing.
- *
- * The execution engine will get an instance of this interface from a data source provider
- * (e.g. {@link org.apache.spark.sql.sources.v2.BatchReadSupportProvider}) at the start of a batch
- * query, then call {@link #newScanConfigBuilder()} and create an instance of {@link ScanConfig}.
- * The {@link ScanConfigBuilder} can apply operator pushdown and keep the pushdown result in
- * {@link ScanConfig}. The {@link ScanConfig} will be used to create input partitions and reader
- * factory to scan data from the data source with a Spark job.
- */
-@InterfaceStability.Evolving
-public interface BatchReadSupport extends ReadSupport {
-
- /**
- * Returns a builder of {@link ScanConfig}. Spark will call this method and create a
- * {@link ScanConfig} for each data scanning job.
- *
- * The builder can take some query specific information to do operators pushdown, and keep these
- * information in the created {@link ScanConfig}.
- *
- * This is the first step of the data scan. All other methods in {@link BatchReadSupport} needs
- * to take {@link ScanConfig} as an input.
- */
- ScanConfigBuilder newScanConfigBuilder();
-
- /**
- * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}.
- */
- PartitionReaderFactory createReaderFactory(ScanConfig config);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java
new file mode 100644
index 0000000..dcb8771
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java
@@ -0,0 +1,35 @@
+/*
+ * 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.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset;
+
+/**
+ * A mix-in interface for {@link InputPartition}. Continuous input partitions can
+ * implement this interface to provide creating {@link InputPartitionReader} with particular offset.
+ */
+@InterfaceStability.Evolving
+public interface ContinuousInputPartition<T> extends InputPartition<T> {
+ /**
+ * Create an input partition reader with particular offset as its startOffset.
+ *
+ * @param offset offset want to set as the input partition reader's startOffset.
+ */
+ InputPartitionReader<T> createContinuousReader(PartitionOffset offset);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java
new file mode 100644
index 0000000..da98fab
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java
@@ -0,0 +1,75 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * A data source reader that is returned by
+ * {@link ReadSupport#createReader(DataSourceOptions)} or
+ * {@link ReadSupport#createReader(StructType, DataSourceOptions)}.
+ * It can mix in various query optimization interfaces to speed up the data scan. The actual scan
+ * logic is delegated to {@link InputPartition}s, which are returned by
+ * {@link #planInputPartitions()}.
+ *
+ * There are mainly 3 kinds of query optimizations:
+ * 1. Operators push-down. E.g., filter push-down, required columns push-down(aka column
+ * pruning), etc. Names of these interfaces start with `SupportsPushDown`.
+ * 2. Information Reporting. E.g., statistics reporting, ordering reporting, etc.
+ * Names of these interfaces start with `SupportsReporting`.
+ * 3. Columnar scan if implements {@link SupportsScanColumnarBatch}.
+ *
+ * If an exception was throw when applying any of these query optimizations, the action will fail
+ * and no Spark job will be submitted.
+ *
+ * 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.
+ */
+@InterfaceStability.Evolving
+public interface DataSourceReader {
+
+ /**
+ * Returns the actual schema of this data source reader, which may be different from the physical
+ * schema of the underlying storage, as column pruning or other optimizations may happen.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ */
+ StructType readSchema();
+
+ /**
+ * Returns a list of {@link InputPartition}s. Each {@link InputPartition} is responsible for
+ * creating a data reader to output data of one RDD partition. The number of input partitions
+ * returned here is the same as the number of RDD partitions this scan outputs.
+ *
+ * Note that, this may not be a full scan if the data source reader mixes in other optimization
+ * interfaces like column pruning, filter push-down, etc. These optimizations are applied before
+ * Spark issues the scan request.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ */
+ List<InputPartition<InternalRow>> planInputPartitions();
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java
index 95c30de..f2038d0 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java
@@ -22,18 +22,18 @@ import java.io.Serializable;
import org.apache.spark.annotation.InterfaceStability;
/**
- * A serializable representation of an input partition returned by
- * {@link ReadSupport#planInputPartitions(ScanConfig)}.
+ * An input partition returned by {@link DataSourceReader#planInputPartitions()} and is
+ * responsible for creating the actual data reader of one RDD partition.
+ * The relationship between {@link InputPartition} and {@link InputPartitionReader}
+ * is similar to the relationship between {@link Iterable} and {@link java.util.Iterator}.
*
- * Note that {@link InputPartition} will be serialized and sent to executors, then
- * {@link PartitionReader} will be created by
- * {@link PartitionReaderFactory#createReader(InputPartition)} or
- * {@link PartitionReaderFactory#createColumnarReader(InputPartition)} on executors to do
- * the actual reading. So {@link InputPartition} must be serializable while {@link PartitionReader}
- * doesn't need to be.
+ * Note that {@link InputPartition}s will be serialized and sent to executors, then
+ * {@link InputPartitionReader}s will be created on executors to do the actual reading. So
+ * {@link InputPartition} must be serializable while {@link InputPartitionReader} doesn't need to
+ * be.
*/
@InterfaceStability.Evolving
-public interface InputPartition extends Serializable {
+public interface InputPartition<T> extends Serializable {
/**
* The preferred locations where the input partition reader returned by this partition can run
@@ -51,4 +51,12 @@ public interface InputPartition extends Serializable {
default String[] preferredLocations() {
return new String[0];
}
+
+ /**
+ * Returns an input partition reader to do the actual reading work.
+ *
+ * If this method fails (by throwing an exception), the corresponding Spark task would fail and
+ * get retried until hitting the maximum retry times.
+ */
+ InputPartitionReader<T> createPartitionReader();
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java
new file mode 100644
index 0000000..f3ff7f5
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java
@@ -0,0 +1,53 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.spark.annotation.InterfaceStability;
+
+/**
+ * An input partition reader returned by {@link InputPartition#createPartitionReader()} and is
+ * responsible for outputting data for a RDD partition.
+ *
+ * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow}
+ * for normal data source readers, {@link org.apache.spark.sql.vectorized.ColumnarBatch} for data
+ * source readers that mix in {@link SupportsScanColumnarBatch}.
+ */
+@InterfaceStability.Evolving
+public interface InputPartitionReader<T> extends Closeable {
+
+ /**
+ * Proceed to next record, returns false if there is no more records.
+ *
+ * If this method fails (by throwing an exception), the corresponding Spark task would fail and
+ * get retried until hitting the maximum retry times.
+ *
+ * @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.
+ *
+ * If this method fails (by throwing an exception), the corresponding Spark task would fail and
+ * get retried until hitting the maximum retry times.
+ */
+ T get();
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java
deleted file mode 100644
index 04ff8d0..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.spark.annotation.InterfaceStability;
-
-/**
- * A partition reader returned by {@link PartitionReaderFactory#createReader(InputPartition)} or
- * {@link PartitionReaderFactory#createColumnarReader(InputPartition)}. It's responsible for
- * outputting data for a RDD partition.
- *
- * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow}
- * for normal data sources, or {@link org.apache.spark.sql.vectorized.ColumnarBatch} for columnar
- * data sources(whose {@link PartitionReaderFactory#supportColumnarReads(InputPartition)}
- * returns true).
- */
-@InterfaceStability.Evolving
-public interface PartitionReader<T> extends Closeable {
-
- /**
- * Proceed to next record, returns false if there is no more records.
- *
- * @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();
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java
deleted file mode 100644
index f35de93..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import java.io.Serializable;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-/**
- * A factory used to create {@link PartitionReader} instances.
- *
- * If Spark fails to execute any methods in the implementations of this interface or in the returned
- * {@link PartitionReader} (by throwing an exception), corresponding Spark task would fail and
- * get retried until hitting the maximum retry times.
- */
-@InterfaceStability.Evolving
-public interface PartitionReaderFactory extends Serializable {
-
- /**
- * Returns a row-based partition reader to read data from the given {@link InputPartition}.
- *
- * Implementations probably need to cast the input partition to the concrete
- * {@link InputPartition} class defined for the data source.
- */
- PartitionReader<InternalRow> createReader(InputPartition partition);
-
- /**
- * Returns a columnar partition reader to read data from the given {@link InputPartition}.
- *
- * Implementations probably need to cast the input partition to the concrete
- * {@link InputPartition} class defined for the data source.
- */
- default PartitionReader<ColumnarBatch> createColumnarReader(InputPartition partition) {
- throw new UnsupportedOperationException("Cannot create columnar reader.");
- }
-
- /**
- * Returns true if the given {@link InputPartition} should be read by Spark in a columnar way.
- * This means, implementations must also implement {@link #createColumnarReader(InputPartition)}
- * for the input partitions that this method returns true.
- *
- * As of Spark 2.4, Spark can only read all input partition in a columnar way, or none of them.
- * Data source can't mix columnar and row-based partitions. This may be relaxed in future
- * versions.
- */
- default boolean supportColumnarReads(InputPartition partition) {
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java
deleted file mode 100644
index a58ddb2..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.types.StructType;
-
-/**
- * The base interface for all the batch and streaming read supports. Data sources should implement
- * concrete read support interfaces like {@link BatchReadSupport}.
- *
- * If Spark fails to execute any methods in the implementations of this interface (by throwing an
- * exception), the read action will fail and no Spark job will be submitted.
- */
-@InterfaceStability.Evolving
-public interface ReadSupport {
-
- /**
- * Returns the full schema of this data source, which is usually the physical schema of the
- * underlying storage. This full schema should not be affected by column pruning or other
- * optimizations.
- */
- StructType fullSchema();
-
- /**
- * Returns a list of {@link InputPartition input partitions}. Each {@link InputPartition}
- * represents a data split that can be processed by one Spark task. The number of input
- * partitions returned here is the same as the number of RDD partitions this scan outputs.
- *
- * Note that, this may not be a full scan if the data source supports optimization like filter
- * push-down. Implementations should check the input {@link ScanConfig} and adjust the resulting
- * {@link InputPartition input partitions}.
- */
- InputPartition[] planInputPartitions(ScanConfig config);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java
deleted file mode 100644
index 7462ce2..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.types.StructType;
-
-/**
- * An interface that carries query specific information for the data scanning job, like operator
- * pushdown information and streaming query offsets. This is defined as an empty interface, and data
- * sources should define their own {@link ScanConfig} classes.
- *
- * For APIs that take a {@link ScanConfig} as input, like
- * {@link ReadSupport#planInputPartitions(ScanConfig)},
- * {@link BatchReadSupport#createReaderFactory(ScanConfig)} and
- * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need to
- * cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source.
- */
-@InterfaceStability.Evolving
-public interface ScanConfig {
-
- /**
- * Returns the actual schema of this data source reader, which may be different from the physical
- * schema of the underlying storage, as column pruning or other optimizations may happen.
- *
- * If this method fails (by throwing an exception), the action will fail and no Spark job will be
- * submitted.
- */
- StructType readSchema();
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java
deleted file mode 100644
index 4c0eedf..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader;
-
-import org.apache.spark.annotation.InterfaceStability;
-
-/**
- * An interface for building the {@link ScanConfig}. Implementations can mixin those
- * SupportsPushDownXYZ interfaces to do operator pushdown, and keep the operator pushdown result in
- * the returned {@link ScanConfig}.
- */
-@InterfaceStability.Evolving
-public interface ScanConfigBuilder {
- ScanConfig build();
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java
index 44799c7..031c7a7 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java
@@ -23,7 +23,7 @@ import org.apache.spark.annotation.InterfaceStability;
/**
* An interface to represent statistics for a data source, which is returned by
- * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}.
+ * {@link SupportsReportStatistics#estimateStatistics()}.
*/
@InterfaceStability.Evolving
public interface Statistics {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
index 5e7985f..7e0020f 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
@@ -21,11 +21,11 @@ import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.sources.Filter;
/**
- * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this interface to
- * push down filters to the data source and reduce the size of the data to be read.
+ * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to push down filters to the data source and reduce the size of the data to be read.
*/
@InterfaceStability.Evolving
-public interface SupportsPushDownFilters extends ScanConfigBuilder {
+public interface SupportsPushDownFilters extends DataSourceReader {
/**
* Pushes down filters, and returns filters that need to be evaluated after scanning.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java
index edb1649..427b4d0 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java
@@ -21,12 +21,12 @@ import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.types.StructType;
/**
- * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this
+ * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this
* interface to push down required columns to the data source and only read these columns during
* scan to reduce the size of the data to be read.
*/
@InterfaceStability.Evolving
-public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder {
+public interface SupportsPushDownRequiredColumns extends DataSourceReader {
/**
* Applies column pruning w.r.t. the given requiredSchema.
@@ -35,8 +35,8 @@ public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder {
* also OK to do the pruning partially, e.g., a data source may not be able to prune nested
* fields, and only prune top-level columns.
*
- * Note that, {@link ScanConfig#readSchema()} implementation should take care of the column
- * pruning applied here.
+ * Note that, data source readers should update {@link DataSourceReader#readSchema()} after
+ * applying column pruning.
*/
void pruneColumns(StructType requiredSchema);
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java
index db62cd4..6b60da7 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java
@@ -21,17 +21,17 @@ import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning;
/**
- * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to
- * report data partitioning and try to avoid shuffle at Spark side.
+ * A mix in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to report data partitioning and try to avoid shuffle at Spark side.
*
- * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition},
- * Spark may avoid adding a shuffle even if the reader does not implement this interface.
+ * Note that, when the reader creates exactly one {@link InputPartition}, Spark may avoid
+ * adding a shuffle even if the reader does not implement this interface.
*/
@InterfaceStability.Evolving
-public interface SupportsReportPartitioning extends ReadSupport {
+public interface SupportsReportPartitioning extends DataSourceReader {
/**
* Returns the output data partitioning that this reader guarantees.
*/
- Partitioning outputPartitioning(ScanConfig config);
+ Partitioning outputPartitioning();
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java
index 1831488..44d0ce3 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java
@@ -20,18 +20,18 @@ package org.apache.spark.sql.sources.v2.reader;
import org.apache.spark.annotation.InterfaceStability;
/**
- * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to
- * report statistics to Spark.
+ * A mix in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to report statistics to Spark.
*
* As of Spark 2.4, statistics are reported to the optimizer before any operator is pushed to the
- * data source. Implementations that return more accurate statistics based on pushed operators will
- * not improve query performance until the planner can push operators before getting stats.
+ * DataSourceReader. Implementations that return more accurate statistics based on pushed operators
+ * will not improve query performance until the planner can push operators before getting stats.
*/
@InterfaceStability.Evolving
-public interface SupportsReportStatistics extends ReadSupport {
+public interface SupportsReportStatistics extends DataSourceReader {
/**
- * Returns the estimated statistics of this data source scan.
+ * Returns the estimated statistics of this data source.
*/
- Statistics estimateStatistics(ScanConfig config);
+ Statistics estimateStatistics();
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java
new file mode 100644
index 0000000..f4da686
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java
@@ -0,0 +1,53 @@
+/*
+ * 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 java.util.List;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+/**
+ * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to output {@link ColumnarBatch} and make the scan faster.
+ */
+@InterfaceStability.Evolving
+public interface SupportsScanColumnarBatch extends DataSourceReader {
+ @Override
+ default List<InputPartition<InternalRow>> planInputPartitions() {
+ throw new IllegalStateException(
+ "planInputPartitions not supported by default within SupportsScanColumnarBatch.");
+ }
+
+ /**
+ * Similar to {@link DataSourceReader#planInputPartitions()}, but returns columnar data
+ * in batches.
+ */
+ List<InputPartition<ColumnarBatch>> planBatchInputPartitions();
+
+ /**
+ * Returns true if the concrete data source reader can read data in batch according to the scan
+ * properties like required columns, pushes filters, etc. It's possible that the implementation
+ * can only support some certain columns with certain types. Users can overwrite this method and
+ * {@link #planInputPartitions()} to fallback to normal read path under some conditions.
+ */
+ default boolean enableBatchRead() {
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java
index 6764d4b..38ca5fc 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java
@@ -18,12 +18,12 @@
package org.apache.spark.sql.sources.v2.reader.partitioning;
import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.sources.v2.reader.PartitionReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
/**
* A concrete implementation of {@link Distribution}. Represents a distribution where records that
* share the same values for the {@link #clusteredColumns} will be produced by the same
- * {@link PartitionReader}.
+ * {@link InputPartitionReader}.
*/
@InterfaceStability.Evolving
public class ClusteredDistribution implements Distribution {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java
index 364a3f5..5e32ba6 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java
@@ -18,14 +18,14 @@
package org.apache.spark.sql.sources.v2.reader.partitioning;
import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.sources.v2.reader.PartitionReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
/**
* An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
+ * be distributed among the data partitions (one {@link InputPartitionReader} outputs data for one
* partition).
* Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
+ * partition(the output records of a single {@link InputPartitionReader}).
*
* The instance of this interface is created and provided by Spark, then consumed by
* {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java
index fb0b6f1..f460f6b 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java
@@ -19,13 +19,12 @@ package org.apache.spark.sql.sources.v2.reader.partitioning;
import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.sources.v2.reader.InputPartition;
-import org.apache.spark.sql.sources.v2.reader.ScanConfig;
import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning;
/**
* An interface to represent the output data partitioning for a data source, which is returned by
- * {@link SupportsReportPartitioning#outputPartitioning(ScanConfig)}. Note that this should work
- * like a snapshot. Once created, it should be deterministic and always report the same number of
+ * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work like a
+ * snapshot. Once created, it should be deterministic and always report the same number of
* partitions and the same "satisfy" result for a certain distribution.
*/
@InterfaceStability.Evolving
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java
new file mode 100644
index 0000000..7b0ba0b
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java
@@ -0,0 +1,36 @@
+/*
+ * 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.streaming;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+
+/**
+ * A variation on {@link InputPartitionReader} for use with streaming in continuous processing mode.
+ */
+@InterfaceStability.Evolving
+public interface ContinuousInputPartitionReader<T> extends InputPartitionReader<T> {
+ /**
+ * 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
+ * as a restart checkpoint.
+ */
+ PartitionOffset getOffset();
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java
deleted file mode 100644
index 9101c8a..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader.streaming;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.sources.v2.reader.PartitionReader;
-
-/**
- * A variation on {@link PartitionReader} for use with continuous streaming processing.
- */
-@InterfaceStability.Evolving
-public interface ContinuousPartitionReader<T> extends PartitionReader<T> {
-
- /**
- * 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
- * as a restart checkpoint.
- */
- PartitionOffset getOffset();
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java
deleted file mode 100644
index 2d9f1ca..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader.streaming;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.sources.v2.reader.InputPartition;
-import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-/**
- * A variation on {@link PartitionReaderFactory} that returns {@link ContinuousPartitionReader}
- * instead of {@link org.apache.spark.sql.sources.v2.reader.PartitionReader}. It's used for
- * continuous streaming processing.
- */
-@InterfaceStability.Evolving
-public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory {
- @Override
- ContinuousPartitionReader<InternalRow> createReader(InputPartition partition);
-
- @Override
- default ContinuousPartitionReader<ColumnarBatch> createColumnarReader(InputPartition partition) {
- throw new UnsupportedOperationException("Cannot create columnar reader.");
- }
-}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[2/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala
deleted file mode 100644
index 5884380..0000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import java.io.ByteArrayOutputStream
-
-import org.apache.spark.sql.execution.streaming.MemoryStream
-import org.apache.spark.sql.streaming.{StreamTest, Trigger}
-
-class ConsoleWriteSupportSuite extends StreamTest {
- import testImplicits._
-
- test("microbatch - default") {
- val input = MemoryStream[Int]
-
- val captured = new ByteArrayOutputStream()
- Console.withOut(captured) {
- val query = input.toDF().writeStream.format("console").start()
- try {
- input.addData(1, 2, 3)
- query.processAllAvailable()
- input.addData(4, 5, 6)
- query.processAllAvailable()
- input.addData()
- query.processAllAvailable()
- } finally {
- query.stop()
- }
- }
-
- assert(captured.toString() ==
- """-------------------------------------------
- |Batch: 0
- |-------------------------------------------
- |+-----+
- ||value|
- |+-----+
- || 1|
- || 2|
- || 3|
- |+-----+
- |
- |-------------------------------------------
- |Batch: 1
- |-------------------------------------------
- |+-----+
- ||value|
- |+-----+
- || 4|
- || 5|
- || 6|
- |+-----+
- |
- |-------------------------------------------
- |Batch: 2
- |-------------------------------------------
- |+-----+
- ||value|
- |+-----+
- |+-----+
- |
- |""".stripMargin)
- }
-
- test("microbatch - with numRows") {
- val input = MemoryStream[Int]
-
- val captured = new ByteArrayOutputStream()
- Console.withOut(captured) {
- val query = input.toDF().writeStream.format("console").option("NUMROWS", 2).start()
- try {
- input.addData(1, 2, 3)
- query.processAllAvailable()
- } finally {
- query.stop()
- }
- }
-
- assert(captured.toString() ==
- """-------------------------------------------
- |Batch: 0
- |-------------------------------------------
- |+-----+
- ||value|
- |+-----+
- || 1|
- || 2|
- |+-----+
- |only showing top 2 rows
- |
- |""".stripMargin)
- }
-
- test("microbatch - truncation") {
- val input = MemoryStream[String]
-
- val captured = new ByteArrayOutputStream()
- Console.withOut(captured) {
- val query = input.toDF().writeStream.format("console").option("TRUNCATE", true).start()
- try {
- input.addData("123456789012345678901234567890")
- query.processAllAvailable()
- } finally {
- query.stop()
- }
- }
-
- assert(captured.toString() ==
- """-------------------------------------------
- |Batch: 0
- |-------------------------------------------
- |+--------------------+
- || value|
- |+--------------------+
- ||12345678901234567...|
- |+--------------------+
- |
- |""".stripMargin)
- }
-
- test("continuous - default") {
- val captured = new ByteArrayOutputStream()
- Console.withOut(captured) {
- val input = spark.readStream
- .format("rate")
- .option("numPartitions", "1")
- .option("rowsPerSecond", "5")
- .load()
- .select('value)
-
- val query = input.writeStream.format("console").trigger(Trigger.Continuous(200)).start()
- assert(query.isActive)
- query.stop()
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala
new file mode 100644
index 0000000..55acf2b
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.sources
+
+import java.io.ByteArrayOutputStream
+
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.streaming.{StreamTest, Trigger}
+
+class ConsoleWriterSuite extends StreamTest {
+ import testImplicits._
+
+ test("microbatch - default") {
+ val input = MemoryStream[Int]
+
+ val captured = new ByteArrayOutputStream()
+ Console.withOut(captured) {
+ val query = input.toDF().writeStream.format("console").start()
+ try {
+ input.addData(1, 2, 3)
+ query.processAllAvailable()
+ input.addData(4, 5, 6)
+ query.processAllAvailable()
+ input.addData()
+ query.processAllAvailable()
+ } finally {
+ query.stop()
+ }
+ }
+
+ assert(captured.toString() ==
+ """-------------------------------------------
+ |Batch: 0
+ |-------------------------------------------
+ |+-----+
+ ||value|
+ |+-----+
+ || 1|
+ || 2|
+ || 3|
+ |+-----+
+ |
+ |-------------------------------------------
+ |Batch: 1
+ |-------------------------------------------
+ |+-----+
+ ||value|
+ |+-----+
+ || 4|
+ || 5|
+ || 6|
+ |+-----+
+ |
+ |-------------------------------------------
+ |Batch: 2
+ |-------------------------------------------
+ |+-----+
+ ||value|
+ |+-----+
+ |+-----+
+ |
+ |""".stripMargin)
+ }
+
+ test("microbatch - with numRows") {
+ val input = MemoryStream[Int]
+
+ val captured = new ByteArrayOutputStream()
+ Console.withOut(captured) {
+ val query = input.toDF().writeStream.format("console").option("NUMROWS", 2).start()
+ try {
+ input.addData(1, 2, 3)
+ query.processAllAvailable()
+ } finally {
+ query.stop()
+ }
+ }
+
+ assert(captured.toString() ==
+ """-------------------------------------------
+ |Batch: 0
+ |-------------------------------------------
+ |+-----+
+ ||value|
+ |+-----+
+ || 1|
+ || 2|
+ |+-----+
+ |only showing top 2 rows
+ |
+ |""".stripMargin)
+ }
+
+ test("microbatch - truncation") {
+ val input = MemoryStream[String]
+
+ val captured = new ByteArrayOutputStream()
+ Console.withOut(captured) {
+ val query = input.toDF().writeStream.format("console").option("TRUNCATE", true).start()
+ try {
+ input.addData("123456789012345678901234567890")
+ query.processAllAvailable()
+ } finally {
+ query.stop()
+ }
+ }
+
+ assert(captured.toString() ==
+ """-------------------------------------------
+ |Batch: 0
+ |-------------------------------------------
+ |+--------------------+
+ || value|
+ |+--------------------+
+ ||12345678901234567...|
+ |+--------------------+
+ |
+ |""".stripMargin)
+ }
+
+ test("continuous - default") {
+ val captured = new ByteArrayOutputStream()
+ Console.withOut(captured) {
+ val input = spark.readStream
+ .format("rate")
+ .option("numPartitions", "1")
+ .option("rowsPerSecond", "5")
+ .load()
+ .select('value)
+
+ val query = input.writeStream.format("console").trigger(Trigger.Continuous(200)).start()
+ assert(query.isActive)
+ query.stop()
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
index dd74af8..5ca13b8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala
@@ -17,18 +17,20 @@
package org.apache.spark.sql.execution.streaming.sources
+import java.util.Optional
import java.util.concurrent.TimeUnit
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous._
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider}
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport}
import org.apache.spark.sql.sources.v2.reader.streaming.Offset
import org.apache.spark.sql.streaming.StreamTest
import org.apache.spark.util.ManualClock
@@ -41,7 +43,7 @@ class RateSourceSuite extends StreamTest {
override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = {
assert(query.nonEmpty)
val rateSource = query.get.logicalPlan.collect {
- case StreamingExecutionRelation(source: RateStreamMicroBatchReadSupport, _) => source
+ case StreamingExecutionRelation(source: RateStreamMicroBatchReader, _) => source
}.head
rateSource.clock.asInstanceOf[ManualClock].advance(TimeUnit.SECONDS.toMillis(seconds))
@@ -54,10 +56,10 @@ class RateSourceSuite extends StreamTest {
test("microbatch in registry") {
withTempDir { temp =>
DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match {
- case ds: MicroBatchReadSupportProvider =>
- val readSupport = ds.createMicroBatchReadSupport(
- temp.getCanonicalPath, DataSourceOptions.empty())
- assert(readSupport.isInstanceOf[RateStreamMicroBatchReadSupport])
+ case ds: MicroBatchReadSupport =>
+ val reader = ds.createMicroBatchReader(
+ Optional.empty(), temp.getCanonicalPath, DataSourceOptions.empty())
+ assert(reader.isInstanceOf[RateStreamMicroBatchReader])
case _ =>
throw new IllegalStateException("Could not find read support for rate")
}
@@ -67,7 +69,7 @@ class RateSourceSuite extends StreamTest {
test("compatible with old path in registry") {
DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.RateSourceProvider",
spark.sqlContext.conf).newInstance() match {
- case ds: MicroBatchReadSupportProvider =>
+ case ds: MicroBatchReadSupport =>
assert(ds.isInstanceOf[RateStreamProvider])
case _ =>
throw new IllegalStateException("Could not find read support for rate")
@@ -139,19 +141,30 @@ class RateSourceSuite extends StreamTest {
)
}
+ test("microbatch - set offset") {
+ withTempDir { temp =>
+ val reader = new RateStreamMicroBatchReader(DataSourceOptions.empty(), temp.getCanonicalPath)
+ val startOffset = LongOffset(0L)
+ val endOffset = LongOffset(1L)
+ reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset))
+ assert(reader.getStartOffset() == startOffset)
+ assert(reader.getEndOffset() == endOffset)
+ }
+ }
+
test("microbatch - infer offsets") {
withTempDir { temp =>
- val readSupport = new RateStreamMicroBatchReadSupport(
+ val reader = new RateStreamMicroBatchReader(
new DataSourceOptions(
Map("numPartitions" -> "1", "rowsPerSecond" -> "100", "useManualClock" -> "true").asJava),
temp.getCanonicalPath)
- readSupport.clock.asInstanceOf[ManualClock].advance(100000)
- val startOffset = readSupport.initialOffset()
- startOffset match {
+ reader.clock.asInstanceOf[ManualClock].advance(100000)
+ reader.setOffsetRange(Optional.empty(), Optional.empty())
+ reader.getStartOffset() match {
case r: LongOffset => assert(r.offset === 0L)
case _ => throw new IllegalStateException("unexpected offset type")
}
- readSupport.latestOffset() match {
+ reader.getEndOffset() match {
case r: LongOffset => assert(r.offset >= 100)
case _ => throw new IllegalStateException("unexpected offset type")
}
@@ -160,16 +173,15 @@ class RateSourceSuite extends StreamTest {
test("microbatch - predetermined batch size") {
withTempDir { temp =>
- val readSupport = new RateStreamMicroBatchReadSupport(
+ val reader = new RateStreamMicroBatchReader(
new DataSourceOptions(Map("numPartitions" -> "1", "rowsPerSecond" -> "20").asJava),
temp.getCanonicalPath)
val startOffset = LongOffset(0L)
val endOffset = LongOffset(1L)
- val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build()
- val tasks = readSupport.planInputPartitions(config)
- val readerFactory = readSupport.createReaderFactory(config)
+ reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset))
+ val tasks = reader.planInputPartitions()
assert(tasks.size == 1)
- val dataReader = readerFactory.createReader(tasks(0))
+ val dataReader = tasks.get(0).createPartitionReader()
val data = ArrayBuffer[InternalRow]()
while (dataReader.next()) {
data.append(dataReader.get())
@@ -180,25 +192,24 @@ class RateSourceSuite extends StreamTest {
test("microbatch - data read") {
withTempDir { temp =>
- val readSupport = new RateStreamMicroBatchReadSupport(
+ val reader = new RateStreamMicroBatchReader(
new DataSourceOptions(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava),
temp.getCanonicalPath)
val startOffset = LongOffset(0L)
val endOffset = LongOffset(1L)
- val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build()
- val tasks = readSupport.planInputPartitions(config)
- val readerFactory = readSupport.createReaderFactory(config)
+ reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset))
+ val tasks = reader.planInputPartitions()
assert(tasks.size == 11)
- val readData = tasks
- .map(readerFactory.createReader)
+ val readData = tasks.asScala
+ .map(_.createPartitionReader())
.flatMap { reader =>
val buf = scala.collection.mutable.ListBuffer[InternalRow]()
while (reader.next()) buf.append(reader.get())
buf
}
- assert(readData.map(_.getLong(1)).sorted === 0.until(33).toArray)
+ assert(readData.map(_.getLong(1)).sorted == Range(0, 33))
}
}
@@ -309,44 +320,41 @@ class RateSourceSuite extends StreamTest {
}
test("user-specified schema given") {
- val exception = intercept[UnsupportedOperationException] {
+ val exception = intercept[AnalysisException] {
spark.readStream
.format("rate")
.schema(spark.range(1).schema)
.load()
}
assert(exception.getMessage.contains(
- "rate source does not support user-specified schema"))
+ "rate source does not support a user-specified schema"))
}
test("continuous in registry") {
DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match {
- case ds: ContinuousReadSupportProvider =>
- val readSupport = ds.createContinuousReadSupport(
- "", DataSourceOptions.empty())
- assert(readSupport.isInstanceOf[RateStreamContinuousReadSupport])
+ case ds: ContinuousReadSupport =>
+ val reader = ds.createContinuousReader(Optional.empty(), "", DataSourceOptions.empty())
+ assert(reader.isInstanceOf[RateStreamContinuousReader])
case _ =>
throw new IllegalStateException("Could not find read support for continuous rate")
}
}
test("continuous data") {
- val readSupport = new RateStreamContinuousReadSupport(
+ val reader = new RateStreamContinuousReader(
new DataSourceOptions(Map("numPartitions" -> "2", "rowsPerSecond" -> "20").asJava))
- val config = readSupport.newScanConfigBuilder(readSupport.initialOffset).build()
- val tasks = readSupport.planInputPartitions(config)
- val readerFactory = readSupport.createContinuousReaderFactory(config)
+ reader.setStartOffset(Optional.empty())
+ val tasks = reader.planInputPartitions()
assert(tasks.size == 2)
val data = scala.collection.mutable.ListBuffer[InternalRow]()
- tasks.foreach {
+ tasks.asScala.foreach {
case t: RateStreamContinuousInputPartition =>
- val startTimeMs = readSupport.initialOffset()
+ val startTimeMs = reader.getStartOffset()
.asInstanceOf[RateStreamOffset]
.partitionToValueAndRunTimeMs(t.partitionIndex)
.runTimeMs
- val r = readerFactory.createReader(t)
- .asInstanceOf[RateStreamContinuousPartitionReader]
+ val r = t.createPartitionReader().asInstanceOf[RateStreamContinuousInputPartitionReader]
for (rowIndex <- 0 to 9) {
r.next()
data.append(r.get())
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
index 409156e..48e5cf7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
@@ -21,6 +21,7 @@ import java.net.{InetSocketAddress, SocketException}
import java.nio.ByteBuffer
import java.nio.channels.ServerSocketChannel
import java.sql.Timestamp
+import java.util.Optional
import java.util.concurrent.LinkedBlockingQueue
import scala.collection.JavaConverters._
@@ -33,8 +34,8 @@ import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupportProvider}
-import org.apache.spark.sql.sources.v2.reader.streaming.Offset
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset}
import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest}
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
@@ -48,9 +49,14 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
serverThread.join()
serverThread = null
}
+ if (batchReader != null) {
+ batchReader.stop()
+ batchReader = null
+ }
}
private var serverThread: ServerThread = null
+ private var batchReader: MicroBatchReader = null
case class AddSocketData(data: String*) extends AddData {
override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = {
@@ -59,7 +65,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
"Cannot add data when there is no query for finding the active socket source")
val sources = query.get.logicalPlan.collect {
- case StreamingExecutionRelation(source: TextSocketMicroBatchReadSupport, _) => source
+ case StreamingExecutionRelation(source: TextSocketMicroBatchReader, _) => source
}
if (sources.isEmpty) {
throw new Exception(
@@ -85,7 +91,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
test("backward compatibility with old path") {
DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.TextSocketSourceProvider",
spark.sqlContext.conf).newInstance() match {
- case ds: MicroBatchReadSupportProvider =>
+ case ds: MicroBatchReadSupport =>
assert(ds.isInstanceOf[TextSocketSourceProvider])
case _ =>
throw new IllegalStateException("Could not find socket source")
@@ -175,16 +181,16 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
test("params not given") {
val provider = new TextSocketSourceProvider
intercept[AnalysisException] {
- provider.createMicroBatchReadSupport(
- "", new DataSourceOptions(Map.empty[String, String].asJava))
+ provider.createMicroBatchReader(Optional.empty(), "",
+ new DataSourceOptions(Map.empty[String, String].asJava))
}
intercept[AnalysisException] {
- provider.createMicroBatchReadSupport(
- "", new DataSourceOptions(Map("host" -> "localhost").asJava))
+ provider.createMicroBatchReader(Optional.empty(), "",
+ new DataSourceOptions(Map("host" -> "localhost").asJava))
}
intercept[AnalysisException] {
- provider.createMicroBatchReadSupport(
- "", new DataSourceOptions(Map("port" -> "1234").asJava))
+ provider.createMicroBatchReader(Optional.empty(), "",
+ new DataSourceOptions(Map("port" -> "1234").asJava))
}
}
@@ -193,7 +199,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
val params = Map("host" -> "localhost", "port" -> "1234", "includeTimestamp" -> "fasle")
intercept[AnalysisException] {
val a = new DataSourceOptions(params.asJava)
- provider.createMicroBatchReadSupport("", a)
+ provider.createMicroBatchReader(Optional.empty(), "", a)
}
}
@@ -203,12 +209,12 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
StructField("name", StringType) ::
StructField("area", StringType) :: Nil)
val params = Map("host" -> "localhost", "port" -> "1234")
- val exception = intercept[UnsupportedOperationException] {
- provider.createMicroBatchReadSupport(
- userSpecifiedSchema, "", new DataSourceOptions(params.asJava))
+ val exception = intercept[AnalysisException] {
+ provider.createMicroBatchReader(
+ Optional.of(userSpecifiedSchema), "", new DataSourceOptions(params.asJava))
}
assert(exception.getMessage.contains(
- "socket source does not support user-specified schema"))
+ "socket source does not support a user-specified schema"))
}
test("input row metrics") {
@@ -299,27 +305,25 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
serverThread = new ServerThread()
serverThread.start()
- val readSupport = new TextSocketContinuousReadSupport(
+ val reader = new TextSocketContinuousReader(
new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost",
"port" -> serverThread.port.toString).asJava))
-
- val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build()
- val tasks = readSupport.planInputPartitions(scanConfig)
+ reader.setStartOffset(Optional.empty())
+ val tasks = reader.planInputPartitions()
assert(tasks.size == 2)
val numRecords = 10
val data = scala.collection.mutable.ListBuffer[Int]()
val offsets = scala.collection.mutable.ListBuffer[Int]()
- val readerFactory = readSupport.createContinuousReaderFactory(scanConfig)
import org.scalatest.time.SpanSugar._
failAfter(5 seconds) {
// inject rows, read and check the data and offsets
for (i <- 0 until numRecords) {
serverThread.enqueue(i.toString)
}
- tasks.foreach {
+ tasks.asScala.foreach {
case t: TextSocketContinuousInputPartition =>
- val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader]
+ val r = t.createPartitionReader().asInstanceOf[TextSocketContinuousInputPartitionReader]
for (i <- 0 until numRecords / 2) {
r.next()
offsets.append(r.getOffset().asInstanceOf[ContinuousRecordPartitionOffset].offset)
@@ -335,15 +339,16 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
data.clear()
case _ => throw new IllegalStateException("Unexpected task type")
}
- assert(readSupport.startOffset.offsets == List(3, 3))
- readSupport.commit(TextSocketOffset(List(5, 5)))
- assert(readSupport.startOffset.offsets == List(5, 5))
+ assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == List(3, 3))
+ reader.commit(TextSocketOffset(List(5, 5)))
+ assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == List(5, 5))
}
def commitOffset(partition: Int, offset: Int): Unit = {
- val offsetsToCommit = readSupport.startOffset.offsets.updated(partition, offset)
- readSupport.commit(TextSocketOffset(offsetsToCommit))
- assert(readSupport.startOffset.offsets == offsetsToCommit)
+ val offsetsToCommit = reader.getStartOffset.asInstanceOf[TextSocketOffset]
+ .offsets.updated(partition, offset)
+ reader.commit(TextSocketOffset(offsetsToCommit))
+ assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == offsetsToCommit)
}
}
@@ -351,13 +356,14 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
serverThread = new ServerThread()
serverThread.start()
- val readSupport = new TextSocketContinuousReadSupport(
+ val reader = new TextSocketContinuousReader(
new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost",
"port" -> serverThread.port.toString).asJava))
-
- readSupport.startOffset = TextSocketOffset(List(5, 5))
+ reader.setStartOffset(Optional.of(TextSocketOffset(List(5, 5))))
+ // ok to commit same offset
+ reader.setStartOffset(Optional.of(TextSocketOffset(List(5, 5))))
assertThrows[IllegalStateException] {
- readSupport.commit(TextSocketOffset(List(6, 6)))
+ reader.commit(TextSocketOffset(List(6, 6)))
}
}
@@ -365,12 +371,12 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
serverThread = new ServerThread()
serverThread.start()
- val readSupport = new TextSocketContinuousReadSupport(
+ val reader = new TextSocketContinuousReader(
new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost",
"includeTimestamp" -> "true",
"port" -> serverThread.port.toString).asJava))
- val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build()
- val tasks = readSupport.planInputPartitions(scanConfig)
+ reader.setStartOffset(Optional.empty())
+ val tasks = reader.planInputPartitions()
assert(tasks.size == 2)
val numRecords = 4
@@ -378,10 +384,9 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before
for (i <- 0 until numRecords) {
serverThread.enqueue(i.toString)
}
- val readerFactory = readSupport.createContinuousReaderFactory(scanConfig)
- tasks.foreach {
+ tasks.asScala.foreach {
case t: TextSocketContinuousInputPartition =>
- val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader]
+ val r = t.createPartitionReader().asInstanceOf[TextSocketContinuousInputPartitionReader]
for (i <- 0 until numRecords / 2) {
r.next()
assert(r.get().get(0, TextSocketReader.SCHEMA_TIMESTAMP)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
index f6c3e0c..12beca2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.sources.v2
+import java.util.{ArrayList, List => JList}
+
import test.org.apache.spark.sql.sources.v2._
import org.apache.spark.SparkException
@@ -36,21 +38,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
class DataSourceV2Suite extends QueryTest with SharedSQLContext {
import testImplicits._
- private def getScanConfig(query: DataFrame): AdvancedScanConfigBuilder = {
- query.queryExecution.executedPlan.collect {
- case d: DataSourceV2ScanExec =>
- d.scanConfig.asInstanceOf[AdvancedScanConfigBuilder]
- }.head
- }
-
- private def getJavaScanConfig(
- query: DataFrame): JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder = {
- query.queryExecution.executedPlan.collect {
- case d: DataSourceV2ScanExec =>
- d.scanConfig.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder]
- }.head
- }
-
test("simplest implementation") {
Seq(classOf[SimpleDataSourceV2], classOf[JavaSimpleDataSourceV2]).foreach { cls =>
withClue(cls.getName) {
@@ -63,6 +50,18 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext {
}
test("advanced implementation") {
+ def getReader(query: DataFrame): AdvancedDataSourceV2#Reader = {
+ query.queryExecution.executedPlan.collect {
+ case d: DataSourceV2ScanExec => d.reader.asInstanceOf[AdvancedDataSourceV2#Reader]
+ }.head
+ }
+
+ def getJavaReader(query: DataFrame): JavaAdvancedDataSourceV2#Reader = {
+ query.queryExecution.executedPlan.collect {
+ case d: DataSourceV2ScanExec => d.reader.asInstanceOf[JavaAdvancedDataSourceV2#Reader]
+ }.head
+ }
+
Seq(classOf[AdvancedDataSourceV2], classOf[JavaAdvancedDataSourceV2]).foreach { cls =>
withClue(cls.getName) {
val df = spark.read.format(cls.getName).load()
@@ -71,58 +70,58 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext {
val q1 = df.select('j)
checkAnswer(q1, (0 until 10).map(i => Row(-i)))
if (cls == classOf[AdvancedDataSourceV2]) {
- val config = getScanConfig(q1)
- assert(config.filters.isEmpty)
- assert(config.requiredSchema.fieldNames === Seq("j"))
+ val reader = getReader(q1)
+ assert(reader.filters.isEmpty)
+ assert(reader.requiredSchema.fieldNames === Seq("j"))
} else {
- val config = getJavaScanConfig(q1)
- assert(config.filters.isEmpty)
- assert(config.requiredSchema.fieldNames === Seq("j"))
+ val reader = getJavaReader(q1)
+ assert(reader.filters.isEmpty)
+ assert(reader.requiredSchema.fieldNames === Seq("j"))
}
val q2 = df.filter('i > 3)
checkAnswer(q2, (4 until 10).map(i => Row(i, -i)))
if (cls == classOf[AdvancedDataSourceV2]) {
- val config = getScanConfig(q2)
- assert(config.filters.flatMap(_.references).toSet == Set("i"))
- assert(config.requiredSchema.fieldNames === Seq("i", "j"))
+ val reader = getReader(q2)
+ assert(reader.filters.flatMap(_.references).toSet == Set("i"))
+ assert(reader.requiredSchema.fieldNames === Seq("i", "j"))
} else {
- val config = getJavaScanConfig(q2)
- assert(config.filters.flatMap(_.references).toSet == Set("i"))
- assert(config.requiredSchema.fieldNames === Seq("i", "j"))
+ val reader = getJavaReader(q2)
+ assert(reader.filters.flatMap(_.references).toSet == Set("i"))
+ assert(reader.requiredSchema.fieldNames === Seq("i", "j"))
}
val q3 = df.select('i).filter('i > 6)
checkAnswer(q3, (7 until 10).map(i => Row(i)))
if (cls == classOf[AdvancedDataSourceV2]) {
- val config = getScanConfig(q3)
- assert(config.filters.flatMap(_.references).toSet == Set("i"))
- assert(config.requiredSchema.fieldNames === Seq("i"))
+ val reader = getReader(q3)
+ assert(reader.filters.flatMap(_.references).toSet == Set("i"))
+ assert(reader.requiredSchema.fieldNames === Seq("i"))
} else {
- val config = getJavaScanConfig(q3)
- assert(config.filters.flatMap(_.references).toSet == Set("i"))
- assert(config.requiredSchema.fieldNames === Seq("i"))
+ val reader = getJavaReader(q3)
+ assert(reader.filters.flatMap(_.references).toSet == Set("i"))
+ assert(reader.requiredSchema.fieldNames === Seq("i"))
}
val q4 = df.select('j).filter('j < -10)
checkAnswer(q4, Nil)
if (cls == classOf[AdvancedDataSourceV2]) {
- val config = getScanConfig(q4)
+ val reader = getReader(q4)
// 'j < 10 is not supported by the testing data source.
- assert(config.filters.isEmpty)
- assert(config.requiredSchema.fieldNames === Seq("j"))
+ assert(reader.filters.isEmpty)
+ assert(reader.requiredSchema.fieldNames === Seq("j"))
} else {
- val config = getJavaScanConfig(q4)
+ val reader = getJavaReader(q4)
// 'j < 10 is not supported by the testing data source.
- assert(config.filters.isEmpty)
- assert(config.requiredSchema.fieldNames === Seq("j"))
+ assert(reader.filters.isEmpty)
+ assert(reader.requiredSchema.fieldNames === Seq("j"))
}
}
}
}
test("columnar batch scan implementation") {
- Seq(classOf[ColumnarDataSourceV2], classOf[JavaColumnarDataSourceV2]).foreach { cls =>
+ Seq(classOf[BatchDataSourceV2], classOf[JavaBatchDataSourceV2]).foreach { cls =>
withClue(cls.getName) {
val df = spark.read.format(cls.getName).load()
checkAnswer(df, (0 until 90).map(i => Row(i, -i)))
@@ -154,25 +153,25 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext {
val df = spark.read.format(cls.getName).load()
checkAnswer(df, Seq(Row(1, 4), Row(1, 4), Row(3, 6), Row(2, 6), Row(4, 2), Row(4, 2)))
- val groupByColA = df.groupBy('i).agg(sum('j))
+ val groupByColA = df.groupBy('a).agg(sum('b))
checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4)))
assert(groupByColA.queryExecution.executedPlan.collectFirst {
case e: ShuffleExchangeExec => e
}.isEmpty)
- val groupByColAB = df.groupBy('i, 'j).agg(count("*"))
+ val groupByColAB = df.groupBy('a, 'b).agg(count("*"))
checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2)))
assert(groupByColAB.queryExecution.executedPlan.collectFirst {
case e: ShuffleExchangeExec => e
}.isEmpty)
- val groupByColB = df.groupBy('j).agg(sum('i))
+ val groupByColB = df.groupBy('b).agg(sum('a))
checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5)))
assert(groupByColB.queryExecution.executedPlan.collectFirst {
case e: ShuffleExchangeExec => e
}.isDefined)
- val groupByAPlusB = df.groupBy('i + 'j).agg(count("*"))
+ val groupByAPlusB = df.groupBy('a + 'b).agg(count("*"))
checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1)))
assert(groupByAPlusB.queryExecution.executedPlan.collectFirst {
case e: ShuffleExchangeExec => e
@@ -273,30 +272,36 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext {
}
test("SPARK-23301: column pruning with arbitrary expressions") {
+ def getReader(query: DataFrame): AdvancedDataSourceV2#Reader = {
+ query.queryExecution.executedPlan.collect {
+ case d: DataSourceV2ScanExec => d.reader.asInstanceOf[AdvancedDataSourceV2#Reader]
+ }.head
+ }
+
val df = spark.read.format(classOf[AdvancedDataSourceV2].getName).load()
val q1 = df.select('i + 1)
checkAnswer(q1, (1 until 11).map(i => Row(i)))
- val config1 = getScanConfig(q1)
- assert(config1.requiredSchema.fieldNames === Seq("i"))
+ val reader1 = getReader(q1)
+ assert(reader1.requiredSchema.fieldNames === Seq("i"))
val q2 = df.select(lit(1))
checkAnswer(q2, (0 until 10).map(i => Row(1)))
- val config2 = getScanConfig(q2)
- assert(config2.requiredSchema.isEmpty)
+ val reader2 = getReader(q2)
+ assert(reader2.requiredSchema.isEmpty)
// 'j === 1 can't be pushed down, but we should still be able do column pruning
val q3 = df.filter('j === -1).select('j * 2)
checkAnswer(q3, Row(-2))
- val config3 = getScanConfig(q3)
- assert(config3.filters.isEmpty)
- assert(config3.requiredSchema.fieldNames === Seq("j"))
+ val reader3 = getReader(q3)
+ assert(reader3.filters.isEmpty)
+ assert(reader3.requiredSchema.fieldNames === Seq("j"))
// column pruning should work with other operators.
val q4 = df.sort('i).limit(1).select('i + 1)
checkAnswer(q4, Row(1))
- val config4 = getScanConfig(q4)
- assert(config4.requiredSchema.fieldNames === Seq("i"))
+ val reader4 = getReader(q4)
+ assert(reader4.requiredSchema.fieldNames === Seq("i"))
}
test("SPARK-23315: get output from canonicalized data source v2 related plans") {
@@ -319,291 +324,240 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext {
}
}
+class SimpleSinglePartitionSource extends DataSourceV2 with ReadSupport {
-case class RangeInputPartition(start: Int, end: Int) extends InputPartition
-
-case class NoopScanConfigBuilder(readSchema: StructType) extends ScanConfigBuilder with ScanConfig {
- override def build(): ScanConfig = this
-}
-
-object SimpleReaderFactory extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val RangeInputPartition(start, end) = partition
- new PartitionReader[InternalRow] {
- private var current = start - 1
-
- override def next(): Boolean = {
- current += 1
- current < end
- }
-
- override def get(): InternalRow = InternalRow(current, -current)
+ class Reader extends DataSourceReader {
+ override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int")
- override def close(): Unit = {}
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
+ java.util.Arrays.asList(new SimpleInputPartition(0, 5))
}
}
+
+ override def createReader(options: DataSourceOptions): DataSourceReader = new Reader
}
-abstract class SimpleReadSupport extends BatchReadSupport {
- override def fullSchema(): StructType = new StructType().add("i", "int").add("j", "int")
+// This class is used by pyspark tests. If this class is modified/moved, make sure pyspark
+// tests still pass.
+class SimpleDataSourceV2 extends DataSourceV2 with ReadSupport {
- override def newScanConfigBuilder(): ScanConfigBuilder = {
- NoopScanConfigBuilder(fullSchema())
- }
+ class Reader extends DataSourceReader {
+ override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int")
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- SimpleReaderFactory
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
+ java.util.Arrays.asList(new SimpleInputPartition(0, 5), new SimpleInputPartition(5, 10))
+ }
}
+
+ override def createReader(options: DataSourceOptions): DataSourceReader = new Reader
}
+class SimpleInputPartition(start: Int, end: Int)
+ extends InputPartition[InternalRow]
+ with InputPartitionReader[InternalRow] {
+ private var current = start - 1
-class SimpleSinglePartitionSource extends DataSourceV2 with BatchReadSupportProvider {
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new SimpleInputPartition(start, end)
- class ReadSupport extends SimpleReadSupport {
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- Array(RangeInputPartition(0, 5))
- }
+ override def next(): Boolean = {
+ current += 1
+ current < end
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport
- }
+ override def get(): InternalRow = InternalRow(current, -current)
+
+ override def close(): Unit = {}
}
-// This class is used by pyspark tests. If this class is modified/moved, make sure pyspark
-// tests still pass.
-class SimpleDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider {
+class AdvancedDataSourceV2 extends DataSourceV2 with ReadSupport {
- class ReadSupport extends SimpleReadSupport {
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10))
- }
- }
+ class Reader extends DataSourceReader
+ with SupportsPushDownRequiredColumns with SupportsPushDownFilters {
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport
- }
-}
+ var requiredSchema = new StructType().add("i", "int").add("j", "int")
+ var filters = Array.empty[Filter]
+ override def pruneColumns(requiredSchema: StructType): Unit = {
+ this.requiredSchema = requiredSchema
+ }
-class AdvancedDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider {
+ override def pushFilters(filters: Array[Filter]): Array[Filter] = {
+ val (supported, unsupported) = filters.partition {
+ case GreaterThan("i", _: Int) => true
+ case _ => false
+ }
+ this.filters = supported
+ unsupported
+ }
- class ReadSupport extends SimpleReadSupport {
- override def newScanConfigBuilder(): ScanConfigBuilder = new AdvancedScanConfigBuilder()
+ override def pushedFilters(): Array[Filter] = filters
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val filters = config.asInstanceOf[AdvancedScanConfigBuilder].filters
+ override def readSchema(): StructType = {
+ requiredSchema
+ }
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
val lowerBound = filters.collectFirst {
case GreaterThan("i", v: Int) => v
}
- val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition]
+ val res = new ArrayList[InputPartition[InternalRow]]
if (lowerBound.isEmpty) {
- res.append(RangeInputPartition(0, 5))
- res.append(RangeInputPartition(5, 10))
+ res.add(new AdvancedInputPartition(0, 5, requiredSchema))
+ res.add(new AdvancedInputPartition(5, 10, requiredSchema))
} else if (lowerBound.get < 4) {
- res.append(RangeInputPartition(lowerBound.get + 1, 5))
- res.append(RangeInputPartition(5, 10))
+ res.add(new AdvancedInputPartition(lowerBound.get + 1, 5, requiredSchema))
+ res.add(new AdvancedInputPartition(5, 10, requiredSchema))
} else if (lowerBound.get < 9) {
- res.append(RangeInputPartition(lowerBound.get + 1, 10))
+ res.add(new AdvancedInputPartition(lowerBound.get + 1, 10, requiredSchema))
}
- res.toArray
- }
-
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- val requiredSchema = config.asInstanceOf[AdvancedScanConfigBuilder].requiredSchema
- new AdvancedReaderFactory(requiredSchema)
+ res
}
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport
- }
+ override def createReader(options: DataSourceOptions): DataSourceReader = new Reader
}
-class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig
- with SupportsPushDownRequiredColumns with SupportsPushDownFilters {
+class AdvancedInputPartition(start: Int, end: Int, requiredSchema: StructType)
+ extends InputPartition[InternalRow] with InputPartitionReader[InternalRow] {
- var requiredSchema = new StructType().add("i", "int").add("j", "int")
- var filters = Array.empty[Filter]
+ private var current = start - 1
- override def pruneColumns(requiredSchema: StructType): Unit = {
- this.requiredSchema = requiredSchema
+ override def createPartitionReader(): InputPartitionReader[InternalRow] = {
+ new AdvancedInputPartition(start, end, requiredSchema)
}
- override def readSchema(): StructType = requiredSchema
+ override def close(): Unit = {}
- override def pushFilters(filters: Array[Filter]): Array[Filter] = {
- val (supported, unsupported) = filters.partition {
- case GreaterThan("i", _: Int) => true
- case _ => false
- }
- this.filters = supported
- unsupported
+ override def next(): Boolean = {
+ current += 1
+ current < end
}
- override def pushedFilters(): Array[Filter] = filters
-
- override def build(): ScanConfig = this
-}
-
-class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val RangeInputPartition(start, end) = partition
- new PartitionReader[InternalRow] {
- private var current = start - 1
-
- override def next(): Boolean = {
- current += 1
- current < end
- }
-
- override def get(): InternalRow = {
- val values = requiredSchema.map(_.name).map {
- case "i" => current
- case "j" => -current
- }
- InternalRow.fromSeq(values)
- }
-
- override def close(): Unit = {}
+ override def get(): InternalRow = {
+ val values = requiredSchema.map(_.name).map {
+ case "i" => current
+ case "j" => -current
}
+ InternalRow.fromSeq(values)
}
}
-class SchemaRequiredDataSource extends DataSourceV2 with BatchReadSupportProvider {
+class SchemaRequiredDataSource extends DataSourceV2 with ReadSupport {
- class ReadSupport(val schema: StructType) extends SimpleReadSupport {
- override def fullSchema(): StructType = schema
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] =
- Array.empty
+ class Reader(val readSchema: StructType) extends DataSourceReader {
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] =
+ java.util.Collections.emptyList()
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
+ override def createReader(options: DataSourceOptions): DataSourceReader = {
throw new IllegalArgumentException("requires a user-supplied schema")
}
- override def createBatchReadSupport(
- schema: StructType, options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport(schema)
+ override def createReader(schema: StructType, options: DataSourceOptions): DataSourceReader = {
+ new Reader(schema)
}
}
-class ColumnarDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider {
+class BatchDataSourceV2 extends DataSourceV2 with ReadSupport {
- class ReadSupport extends SimpleReadSupport {
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- Array(RangeInputPartition(0, 50), RangeInputPartition(50, 90))
- }
+ class Reader extends DataSourceReader with SupportsScanColumnarBatch {
+ override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int")
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- ColumnarReaderFactory
+ override def planBatchInputPartitions(): JList[InputPartition[ColumnarBatch]] = {
+ java.util.Arrays.asList(
+ new BatchInputPartitionReader(0, 50), new BatchInputPartitionReader(50, 90))
}
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport
- }
+ override def createReader(options: DataSourceOptions): DataSourceReader = new Reader
}
-object ColumnarReaderFactory extends PartitionReaderFactory {
- private final val BATCH_SIZE = 20
+class BatchInputPartitionReader(start: Int, end: Int)
+ extends InputPartition[ColumnarBatch] with InputPartitionReader[ColumnarBatch] {
- override def supportColumnarReads(partition: InputPartition): Boolean = true
+ private final val BATCH_SIZE = 20
+ private lazy val i = new OnHeapColumnVector(BATCH_SIZE, IntegerType)
+ private lazy val j = new OnHeapColumnVector(BATCH_SIZE, IntegerType)
+ private lazy val batch = new ColumnarBatch(Array(i, j))
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- throw new UnsupportedOperationException
- }
+ private var current = start
- override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = {
- val RangeInputPartition(start, end) = partition
- new PartitionReader[ColumnarBatch] {
- private lazy val i = new OnHeapColumnVector(BATCH_SIZE, IntegerType)
- private lazy val j = new OnHeapColumnVector(BATCH_SIZE, IntegerType)
- private lazy val batch = new ColumnarBatch(Array(i, j))
-
- private var current = start
-
- override def next(): Boolean = {
- i.reset()
- j.reset()
-
- var count = 0
- while (current < end && count < BATCH_SIZE) {
- i.putInt(count, current)
- j.putInt(count, -current)
- current += 1
- count += 1
- }
+ override def createPartitionReader(): InputPartitionReader[ColumnarBatch] = this
- if (count == 0) {
- false
- } else {
- batch.setNumRows(count)
- true
- }
- }
+ override def next(): Boolean = {
+ i.reset()
+ j.reset()
- override def get(): ColumnarBatch = batch
+ var count = 0
+ while (current < end && count < BATCH_SIZE) {
+ i.putInt(count, current)
+ j.putInt(count, -current)
+ current += 1
+ count += 1
+ }
- override def close(): Unit = batch.close()
+ if (count == 0) {
+ false
+ } else {
+ batch.setNumRows(count)
+ true
}
}
+
+ override def get(): ColumnarBatch = {
+ batch
+ }
+
+ override def close(): Unit = batch.close()
}
+class PartitionAwareDataSource extends DataSourceV2 with ReadSupport {
-class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvider {
+ class Reader extends DataSourceReader with SupportsReportPartitioning {
+ override def readSchema(): StructType = new StructType().add("a", "int").add("b", "int")
- class ReadSupport extends SimpleReadSupport with SupportsReportPartitioning {
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
// Note that we don't have same value of column `a` across partitions.
- Array(
- SpecificInputPartition(Array(1, 1, 3), Array(4, 4, 6)),
- SpecificInputPartition(Array(2, 4, 4), Array(6, 2, 2)))
- }
-
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- SpecificReaderFactory
+ java.util.Arrays.asList(
+ new SpecificInputPartitionReader(Array(1, 1, 3), Array(4, 4, 6)),
+ new SpecificInputPartitionReader(Array(2, 4, 4), Array(6, 2, 2)))
}
- override def outputPartitioning(config: ScanConfig): Partitioning = new MyPartitioning
+ override def outputPartitioning(): Partitioning = new MyPartitioning
}
class MyPartitioning extends Partitioning {
override def numPartitions(): Int = 2
override def satisfy(distribution: Distribution): Boolean = distribution match {
- case c: ClusteredDistribution => c.clusteredColumns.contains("i")
+ case c: ClusteredDistribution => c.clusteredColumns.contains("a")
case _ => false
}
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
- new ReadSupport
- }
+ override def createReader(options: DataSourceOptions): DataSourceReader = new Reader
}
-case class SpecificInputPartition(i: Array[Int], j: Array[Int]) extends InputPartition
+class SpecificInputPartitionReader(i: Array[Int], j: Array[Int])
+ extends InputPartition[InternalRow]
+ with InputPartitionReader[InternalRow] {
+ assert(i.length == j.length)
-object SpecificReaderFactory extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[SpecificInputPartition]
- new PartitionReader[InternalRow] {
- private var current = -1
+ private var current = -1
- override def next(): Boolean = {
- current += 1
- current < p.i.length
- }
+ override def createPartitionReader(): InputPartitionReader[InternalRow] = this
- override def get(): InternalRow = InternalRow(p.i(current), p.j(current))
-
- override def close(): Unit = {}
- }
+ override def next(): Boolean = {
+ current += 1
+ current < i.length
}
+
+ override def get(): InternalRow = InternalRow(i(current), j(current))
+
+ override def close(): Unit = {}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
index 952241b..e1b8e9c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala
@@ -18,36 +18,34 @@
package org.apache.spark.sql.sources.v2
import java.io.{BufferedReader, InputStreamReader, IOException}
-import java.util.Optional
+import java.util.{Collections, List => JList, Optional}
import scala.collection.JavaConverters._
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path}
import org.apache.spark.SparkContext
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, InputPartition, InputPartitionReader}
import org.apache.spark.sql.sources.v2.writer._
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.util.SerializableConfiguration
/**
* A HDFS based transactional writable data source.
- * Each task writes data to `target/_temporary/queryId/$jobId-$partitionId-$attemptNumber`.
- * Each job moves files from `target/_temporary/queryId/` to `target`.
+ * Each task writes data to `target/_temporary/jobId/$jobId-$partitionId-$attemptNumber`.
+ * Each job moves files from `target/_temporary/jobId/` to `target`.
*/
-class SimpleWritableDataSource extends DataSourceV2
- with BatchReadSupportProvider with BatchWriteSupportProvider {
+class SimpleWritableDataSource extends DataSourceV2 with ReadSupport with WriteSupport {
private val schema = new StructType().add("i", "long").add("j", "long")
- class ReadSupport(path: String, conf: Configuration) extends SimpleReadSupport {
+ class Reader(path: String, conf: Configuration) extends DataSourceReader {
+ override def readSchema(): StructType = schema
- override def fullSchema(): StructType = schema
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
val dataPath = new Path(path)
val fs = dataPath.getFileSystem(conf)
if (fs.exists(dataPath)) {
@@ -55,23 +53,21 @@ class SimpleWritableDataSource extends DataSourceV2
val name = status.getPath.getName
name.startsWith("_") || name.startsWith(".")
}.map { f =>
- CSVInputPartitionReader(f.getPath.toUri.toString)
- }.toArray
+ val serializableConf = new SerializableConfiguration(conf)
+ new SimpleCSVInputPartitionReader(
+ f.getPath.toUri.toString,
+ serializableConf): InputPartition[InternalRow]
+ }.toList.asJava
} else {
- Array.empty
+ Collections.emptyList()
}
}
-
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- val serializableConf = new SerializableConfiguration(conf)
- new CSVReaderFactory(serializableConf)
- }
}
- class WritSupport(queryId: String, path: String, conf: Configuration) extends BatchWriteSupport {
- override def createBatchWriterFactory(): DataWriterFactory = {
+ class Writer(jobId: String, path: String, conf: Configuration) extends DataSourceWriter {
+ override def createWriterFactory(): DataWriterFactory[InternalRow] = {
SimpleCounter.resetCounter
- new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf))
+ new CSVDataWriterFactory(path, jobId, new SerializableConfiguration(conf))
}
override def onDataWriterCommit(message: WriterCommitMessage): Unit = {
@@ -80,7 +76,7 @@ class SimpleWritableDataSource extends DataSourceV2
override def commit(messages: Array[WriterCommitMessage]): Unit = {
val finalPath = new Path(path)
- val jobPath = new Path(new Path(finalPath, "_temporary"), queryId)
+ val jobPath = new Path(new Path(finalPath, "_temporary"), jobId)
val fs = jobPath.getFileSystem(conf)
try {
for (file <- fs.listStatus(jobPath).map(_.getPath)) {
@@ -95,23 +91,23 @@ class SimpleWritableDataSource extends DataSourceV2
}
override def abort(messages: Array[WriterCommitMessage]): Unit = {
- val jobPath = new Path(new Path(path, "_temporary"), queryId)
+ val jobPath = new Path(new Path(path, "_temporary"), jobId)
val fs = jobPath.getFileSystem(conf)
fs.delete(jobPath, true)
}
}
- override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = {
+ override def createReader(options: DataSourceOptions): DataSourceReader = {
val path = new Path(options.get("path").get())
val conf = SparkContext.getActive.get.hadoopConfiguration
- new ReadSupport(path.toUri.toString, conf)
+ new Reader(path.toUri.toString, conf)
}
- override def createBatchWriteSupport(
- queryId: String,
+ override def createWriter(
+ jobId: String,
schema: StructType,
mode: SaveMode,
- options: DataSourceOptions): Optional[BatchWriteSupport] = {
+ options: DataSourceOptions): Optional[DataSourceWriter] = {
assert(DataType.equalsStructurally(schema.asNullable, this.schema.asNullable))
assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false))
@@ -134,42 +130,39 @@ class SimpleWritableDataSource extends DataSourceV2
}
val pathStr = path.toUri.toString
- Optional.of(new WritSupport(queryId, pathStr, conf))
+ Optional.of(new Writer(jobId, pathStr, conf))
}
}
-case class CSVInputPartitionReader(path: String) extends InputPartition
+class SimpleCSVInputPartitionReader(path: String, conf: SerializableConfiguration)
+ extends InputPartition[InternalRow] with InputPartitionReader[InternalRow] {
-class CSVReaderFactory(conf: SerializableConfiguration)
- extends PartitionReaderFactory {
+ @transient private var lines: Iterator[String] = _
+ @transient private var currentLine: String = _
+ @transient private var inputStream: FSDataInputStream = _
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val path = partition.asInstanceOf[CSVInputPartitionReader].path
+ override def createPartitionReader(): InputPartitionReader[InternalRow] = {
val filePath = new Path(path)
val fs = filePath.getFileSystem(conf.value)
+ inputStream = fs.open(filePath)
+ lines = new BufferedReader(new InputStreamReader(inputStream))
+ .lines().iterator().asScala
+ this
+ }
- new PartitionReader[InternalRow] {
- private val inputStream = fs.open(filePath)
- private val lines = new BufferedReader(new InputStreamReader(inputStream))
- .lines().iterator().asScala
-
- private var currentLine: String = _
-
- override def next(): Boolean = {
- if (lines.hasNext) {
- currentLine = lines.next()
- true
- } else {
- false
- }
- }
+ override def next(): Boolean = {
+ if (lines.hasNext) {
+ currentLine = lines.next()
+ true
+ } else {
+ false
+ }
+ }
- override def get(): InternalRow = InternalRow(currentLine.split(",").map(_.trim.toLong): _*)
+ override def get(): InternalRow = InternalRow(currentLine.split(",").map(_.trim.toLong): _*)
- override def close(): Unit = {
- inputStream.close()
- }
- }
+ override def close(): Unit = {
+ inputStream.close()
}
}
@@ -190,11 +183,12 @@ private[v2] object SimpleCounter {
}
class CSVDataWriterFactory(path: String, jobId: String, conf: SerializableConfiguration)
- extends DataWriterFactory {
+ extends DataWriterFactory[InternalRow] {
- override def createWriter(
+ override def createDataWriter(
partitionId: Int,
- taskId: Long): DataWriter[InternalRow] = {
+ taskId: Long,
+ epochId: Long): DataWriter[InternalRow] = {
val jobPath = new Path(new Path(path, "_temporary"), jobId)
val filePath = new Path(jobPath, s"$jobId-$partitionId-$taskId")
val fs = filePath.getFileSystem(conf.value)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
index 491dc34..35644c5 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
@@ -686,7 +686,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be
plan
.collect {
case r: StreamingExecutionRelation => r.source
- case r: StreamingDataSourceV2Relation => r.readSupport
+ case r: StreamingDataSourceV2Relation => r.reader
}
.zipWithIndex
.find(_._1 == source)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala
----------------------------------------------------------------------
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 fe77a1b..0f15cd6 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
@@ -299,9 +299,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
try {
val input = new MemoryStream[Int](0, sqlContext) {
@volatile var numTriggers = 0
- override def latestOffset(): OffsetV2 = {
+ override def getEndOffset: OffsetV2 = {
numTriggers += 1
- super.latestOffset()
+ super.getEndOffset
}
}
val clock = new StreamManualClock()
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 1dd8175..0278e2a 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
@@ -17,6 +17,8 @@
package org.apache.spark.sql.streaming
+import java.{util => ju}
+import java.util.Optional
import java.util.concurrent.CountDownLatch
import scala.collection.mutable
@@ -30,12 +32,13 @@ import org.scalatest.mockito.MockitoSugar
import org.apache.spark.SparkException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Column, DataFrame, Dataset, Row}
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.sources.TestForeachWriter
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig}
+import org.apache.spark.sql.sources.v2.reader.InputPartition
import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2}
import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock}
import org.apache.spark.sql.types.StructType
@@ -212,17 +215,25 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
private def dataAdded: Boolean = currentOffset.offset != -1
- // latestOffset should take 50 ms the first time it is called after data is added
- override def latestOffset(): OffsetV2 = synchronized {
- if (dataAdded) clock.waitTillTime(1050)
- super.latestOffset()
+ // setOffsetRange should take 50 ms the first time it is called after data is added
+ override def setOffsetRange(start: Optional[OffsetV2], end: Optional[OffsetV2]): Unit = {
+ synchronized {
+ if (dataAdded) clock.waitTillTime(1050)
+ super.setOffsetRange(start, end)
+ }
+ }
+
+ // getEndOffset should take 100 ms the first time it is called after data is added
+ override def getEndOffset(): OffsetV2 = synchronized {
+ if (dataAdded) clock.waitTillTime(1150)
+ super.getEndOffset()
}
// getBatch should take 100 ms the first time it is called
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
+ override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
synchronized {
- clock.waitTillTime(1150)
- super.planInputPartitions(config)
+ clock.waitTillTime(1350)
+ super.planInputPartitions()
}
}
}
@@ -263,26 +274,34 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
AssertOnQuery(_.status.message === "Waiting for next trigger"),
AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0),
- // Test status and progress when `latestOffset` is being called
+ // Test status and progress when setOffsetRange is being called
AddData(inputData, 1, 2),
- AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on `latestOffset`
+ AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on setOffsetRange
AssertStreamExecThreadIsWaitingForTime(1050),
AssertOnQuery(_.status.isDataAvailable === false),
AssertOnQuery(_.status.isTriggerActive === true),
AssertOnQuery(_.status.message.startsWith("Getting offsets from")),
AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0),
- AdvanceManualClock(50), // time = 1050 to unblock `latestOffset`
+ AdvanceManualClock(50), // time = 1050 to unblock setOffsetRange
AssertClockTime(1050),
- // will block on `planInputPartitions` that needs 1350
- AssertStreamExecThreadIsWaitingForTime(1150),
+ AssertStreamExecThreadIsWaitingForTime(1150), // will block on getEndOffset that needs 1150
+ AssertOnQuery(_.status.isDataAvailable === false),
+ AssertOnQuery(_.status.isTriggerActive === true),
+ AssertOnQuery(_.status.message.startsWith("Getting offsets from")),
+ AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0),
+
+ AdvanceManualClock(100), // time = 1150 to unblock getEndOffset
+ AssertClockTime(1150),
+ // will block on planInputPartitions that needs 1350
+ AssertStreamExecThreadIsWaitingForTime(1350),
AssertOnQuery(_.status.isDataAvailable === true),
AssertOnQuery(_.status.isTriggerActive === true),
AssertOnQuery(_.status.message === "Processing new data"),
AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0),
- AdvanceManualClock(100), // time = 1150 to unblock `planInputPartitions`
- AssertClockTime(1150),
+ AdvanceManualClock(200), // time = 1350 to unblock planInputPartitions
+ AssertClockTime(1350),
AssertStreamExecThreadIsWaitingForTime(1500), // will block on map task that needs 1500
AssertOnQuery(_.status.isDataAvailable === true),
AssertOnQuery(_.status.isTriggerActive === true),
@@ -290,7 +309,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0),
// Test status and progress while batch processing has completed
- AdvanceManualClock(350), // time = 1500 to unblock map task
+ AdvanceManualClock(150), // time = 1500 to unblock map task
AssertClockTime(1500),
CheckAnswer(2),
AssertStreamExecThreadIsWaitingForTime(2000), // will block until the next trigger
@@ -310,10 +329,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
assert(progress.numInputRows === 2)
assert(progress.processedRowsPerSecond === 4.0)
- assert(progress.durationMs.get("latestOffset") === 50)
- assert(progress.durationMs.get("queryPlanning") === 100)
+ assert(progress.durationMs.get("setOffsetRange") === 50)
+ assert(progress.durationMs.get("getEndOffset") === 100)
+ assert(progress.durationMs.get("queryPlanning") === 200)
assert(progress.durationMs.get("walCommit") === 0)
- assert(progress.durationMs.get("addBatch") === 350)
+ assert(progress.durationMs.get("addBatch") === 150)
assert(progress.durationMs.get("triggerExecution") === 500)
assert(progress.sources.length === 1)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala
index d6819ea..4f19881 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala
@@ -22,15 +22,16 @@ import java.util.concurrent.{ArrayBlockingQueue, BlockingQueue}
import org.mockito.Mockito._
import org.scalatest.mockito.MockitoSugar
-import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.{SparkEnv, SparkFunSuite, TaskContext}
+import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.execution.streaming.continuous._
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousReadSupport, PartitionOffset}
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.reader.InputPartition
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, PartitionOffset}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.StreamTest
-import org.apache.spark.sql.types.{DataType, IntegerType, StructType}
+import org.apache.spark.sql.types.{DataType, IntegerType}
class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar {
case class LongPartitionOffset(offset: Long) extends PartitionOffset
@@ -43,8 +44,8 @@ class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar {
override def beforeEach(): Unit = {
super.beforeEach()
epochEndpoint = EpochCoordinatorRef.create(
- mock[StreamingWriteSupport],
- mock[ContinuousReadSupport],
+ mock[StreamWriter],
+ mock[ContinuousReader],
mock[ContinuousExecution],
coordinatorId,
startEpoch,
@@ -72,26 +73,26 @@ class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar {
*/
private def setup(): (BlockingQueue[UnsafeRow], ContinuousQueuedDataReader) = {
val queue = new ArrayBlockingQueue[UnsafeRow](1024)
- val partitionReader = new ContinuousPartitionReader[InternalRow] {
- var index = -1
- var curr: UnsafeRow = _
-
- override def next() = {
- curr = queue.take()
- index += 1
- true
- }
+ val factory = new InputPartition[InternalRow] {
+ override def createPartitionReader() = new ContinuousInputPartitionReader[InternalRow] {
+ var index = -1
+ var curr: UnsafeRow = _
+
+ override def next() = {
+ curr = queue.take()
+ index += 1
+ true
+ }
- override def get = curr
+ override def get = curr
- override def getOffset = LongPartitionOffset(index)
+ override def getOffset = LongPartitionOffset(index)
- override def close() = {}
+ override def close() = {}
+ }
}
val reader = new ContinuousQueuedDataReader(
- 0,
- partitionReader,
- new StructType().add("i", "int"),
+ new ContinuousDataSourceRDDPartition(0, factory),
mockContext,
dataQueueSize = sqlContext.conf.continuousStreamingExecutorQueueSize,
epochPollIntervalMs = sqlContext.conf.continuousStreamingExecutorPollIntervalMs)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
index 3d21bc6..4980b0c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala
@@ -41,7 +41,7 @@ class ContinuousSuiteBase extends StreamTest {
case s: ContinuousExecution =>
assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized")
val reader = s.lastExecution.executedPlan.collectFirst {
- case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReadSupport, _) => r
+ case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReader) => r
}.get
val deltaMs = numTriggers * 1000 + 300
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
index 3c973d8..82836dc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala
@@ -27,9 +27,9 @@ import org.apache.spark._
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.sql.LocalSparkSession
import org.apache.spark.sql.execution.streaming.continuous._
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset}
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.test.TestSparkSession
class EpochCoordinatorSuite
@@ -40,20 +40,20 @@ class EpochCoordinatorSuite
private var epochCoordinator: RpcEndpointRef = _
- private var writeSupport: StreamingWriteSupport = _
+ private var writer: StreamWriter = _
private var query: ContinuousExecution = _
private var orderVerifier: InOrder = _
override def beforeEach(): Unit = {
- val reader = mock[ContinuousReadSupport]
- writeSupport = mock[StreamingWriteSupport]
+ val reader = mock[ContinuousReader]
+ writer = mock[StreamWriter]
query = mock[ContinuousExecution]
- orderVerifier = inOrder(writeSupport, query)
+ orderVerifier = inOrder(writer, query)
spark = new TestSparkSession()
epochCoordinator
- = EpochCoordinatorRef.create(writeSupport, reader, query, "test", 1, spark, SparkEnv.get)
+ = EpochCoordinatorRef.create(writer, reader, query, "test", 1, spark, SparkEnv.get)
}
test("single epoch") {
@@ -209,12 +209,12 @@ class EpochCoordinatorSuite
}
private def verifyCommit(epoch: Long): Unit = {
- orderVerifier.verify(writeSupport).commit(eqTo(epoch), any())
+ orderVerifier.verify(writer).commit(eqTo(epoch), any())
orderVerifier.verify(query).commit(epoch)
}
private def verifyNoCommitFor(epoch: Long): Unit = {
- verify(writeSupport, never()).commit(eqTo(epoch), any())
+ verify(writer, never()).commit(eqTo(epoch), any())
verify(query, never()).commit(epoch)
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[5/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java
deleted file mode 100644
index 9a3ad2e..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader.streaming;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.streaming.BaseStreamingSource;
-import org.apache.spark.sql.sources.v2.reader.InputPartition;
-import org.apache.spark.sql.sources.v2.reader.ScanConfig;
-import org.apache.spark.sql.sources.v2.reader.ScanConfigBuilder;
-
-/**
- * An interface that defines how to load the data from data source for continuous streaming
- * processing.
- *
- * The execution engine will get an instance of this interface from a data source provider
- * (e.g. {@link org.apache.spark.sql.sources.v2.ContinuousReadSupportProvider}) at the start of a
- * streaming query, then call {@link #newScanConfigBuilder(Offset)} and create an instance of
- * {@link ScanConfig} for the duration of the streaming query or until
- * {@link #needsReconfiguration(ScanConfig)} is true. The {@link ScanConfig} will be used to create
- * input partitions and reader factory to scan data with a Spark job for its duration. At the end
- * {@link #stop()} will be called when the streaming execution is completed. Note that a single
- * query may have multiple executions due to restart or failure recovery.
- */
-@InterfaceStability.Evolving
-public interface ContinuousReadSupport extends StreamingReadSupport, BaseStreamingSource {
-
- /**
- * Returns a builder of {@link ScanConfig}. Spark will call this method and create a
- * {@link ScanConfig} for each data scanning job.
- *
- * The builder can take some query specific information to do operators pushdown, store streaming
- * offsets, etc., and keep these information in the created {@link ScanConfig}.
- *
- * This is the first step of the data scan. All other methods in {@link ContinuousReadSupport}
- * needs to take {@link ScanConfig} as an input.
- */
- ScanConfigBuilder newScanConfigBuilder(Offset start);
-
- /**
- * Returns a factory, which produces one {@link ContinuousPartitionReader} for one
- * {@link InputPartition}.
- */
- ContinuousPartitionReaderFactory createContinuousReaderFactory(ScanConfig config);
-
- /**
- * Merge partitioned offsets coming from {@link ContinuousPartitionReader} instances
- * for each partition to a single global offset.
- */
- Offset mergeOffsets(PartitionOffset[] offsets);
-
- /**
- * The execution engine will call this method in every epoch to determine if new input
- * partitions 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 {@link ContinuousReadSupport}
- * instance.
- */
- default boolean needsReconfiguration(ScanConfig config) {
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java
new file mode 100644
index 0000000..6e960be
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java
@@ -0,0 +1,79 @@
+/*
+ * 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.streaming;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.execution.streaming.BaseStreamingSource;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+
+import java.util.Optional;
+
+/**
+ * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to allow reading in a continuous processing mode stream.
+ *
+ * Implementations must ensure each partition reader is a {@link ContinuousInputPartitionReader}.
+ *
+ * Note: This class currently extends {@link BaseStreamingSource} to maintain compatibility with
+ * DataSource V1 APIs. This extension will be removed once we get rid of V1 completely.
+ */
+@InterfaceStability.Evolving
+public interface ContinuousReader extends BaseStreamingSource, DataSourceReader {
+ /**
+ * Merge partitioned offsets coming from {@link ContinuousInputPartitionReader} instances
+ * for each partition to a single global offset.
+ */
+ 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 deserializeOffset(String json);
+
+ /**
+ * Set the desired start offset for partitions created from this reader. The scan will
+ * start from the first record after the provided offset, or from an implementation-defined
+ * inferred starting point if no offset is provided.
+ */
+ void setStartOffset(Optional<Offset> start);
+
+ /**
+ * Return the specified or inferred start offset for this reader.
+ *
+ * @throws IllegalStateException if setStartOffset has not been called
+ */
+ Offset getStartOffset();
+
+ /**
+ * The execution engine will call this method in every epoch to determine if new input
+ * partitions 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;
+ }
+
+ /**
+ * 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);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java
deleted file mode 100644
index edb0db1..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader.streaming;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.execution.streaming.BaseStreamingSource;
-import org.apache.spark.sql.sources.v2.reader.*;
-
-/**
- * An interface that defines how to scan the data from data source for micro-batch streaming
- * processing.
- *
- * The execution engine will get an instance of this interface from a data source provider
- * (e.g. {@link org.apache.spark.sql.sources.v2.MicroBatchReadSupportProvider}) at the start of a
- * streaming query, then call {@link #newScanConfigBuilder(Offset, Offset)} and create an instance
- * of {@link ScanConfig} for each micro-batch. The {@link ScanConfig} will be used to create input
- * partitions and reader factory to scan a micro-batch with a Spark job. At the end {@link #stop()}
- * will be called when the streaming execution is completed. Note that a single query may have
- * multiple executions due to restart or failure recovery.
- */
-@InterfaceStability.Evolving
-public interface MicroBatchReadSupport extends StreamingReadSupport, BaseStreamingSource {
-
- /**
- * Returns a builder of {@link ScanConfig}. Spark will call this method and create a
- * {@link ScanConfig} for each data scanning job.
- *
- * The builder can take some query specific information to do operators pushdown, store streaming
- * offsets, etc., and keep these information in the created {@link ScanConfig}.
- *
- * This is the first step of the data scan. All other methods in {@link MicroBatchReadSupport}
- * needs to take {@link ScanConfig} as an input.
- */
- ScanConfigBuilder newScanConfigBuilder(Offset start, Offset end);
-
- /**
- * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}.
- */
- PartitionReaderFactory createReaderFactory(ScanConfig config);
-
- /**
- * Returns the most recent offset available.
- */
- Offset latestOffset();
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java
new file mode 100644
index 0000000..0159c73
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java
@@ -0,0 +1,75 @@
+/*
+ * 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.streaming;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.execution.streaming.BaseStreamingSource;
+
+import java.util.Optional;
+
+/**
+ * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this
+ * interface to indicate they allow micro-batch streaming reads.
+ *
+ * Note: This class currently extends {@link BaseStreamingSource} to maintain compatibility with
+ * DataSource V1 APIs. This extension will be removed once we get rid of V1 completely.
+ */
+@InterfaceStability.Evolving
+public interface MicroBatchReader extends DataSourceReader, BaseStreamingSource {
+ /**
+ * Set the desired offset range for input partitions created from this reader. Partition readers
+ * 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 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 endpoint, such as the last available offset
+ * or the start offset plus a target batch size.
+ */
+ void setOffsetRange(Optional<Offset> start, Optional<Offset> end);
+
+ /**
+ * 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 (if explicitly set through setOffsetRange) or inferred end offset
+ * for this reader.
+ *
+ * @throws IllegalStateException if setOffsetRange has not been called
+ */
+ Offset getEndOffset();
+
+ /**
+ * 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 deserializeOffset(String json);
+
+ /**
+ * 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);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java
index 6cf2773..e41c035 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java
@@ -20,8 +20,8 @@ package org.apache.spark.sql.sources.v2.reader.streaming;
import org.apache.spark.annotation.InterfaceStability;
/**
- * An abstract representation of progress through a {@link MicroBatchReadSupport} or
- * {@link ContinuousReadSupport}.
+ * An abstract representation of progress through a {@link MicroBatchReader} or
+ * {@link ContinuousReader}.
* During execution, offsets provided by the data source implementation will be logged and used as
* restart checkpoints. Each source should provide an offset implementation which the source can use
* to reconstruct a position in the stream up to which data has been seen/processed.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java
deleted file mode 100644
index 84872d1..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.reader.streaming;
-
-import org.apache.spark.sql.sources.v2.reader.ReadSupport;
-
-/**
- * A base interface for streaming read support. This is package private and is invisible to data
- * sources. Data sources should implement concrete streaming read support interfaces:
- * {@link MicroBatchReadSupport} or {@link ContinuousReadSupport}.
- */
-interface StreamingReadSupport extends ReadSupport {
-
- /**
- * Returns the initial offset for a streaming query to start reading from. Note that the
- * streaming data source should not assume that it will start reading from its initial offset:
- * if Spark is restarting an existing query, it will restart from the check-pointed offset rather
- * than the initial one.
- */
- Offset initialOffset();
-
- /**
- * 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 deserializeOffset(String json);
-
- /**
- * 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);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java
deleted file mode 100644
index 0ec9e05..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.writer;
-
-import org.apache.spark.annotation.InterfaceStability;
-
-/**
- * An interface that defines how to write the data to data source for batch processing.
- *
- * The writing procedure is:
- * 1. Create a writer factory by {@link #createBatchWriterFactory()}, serialize and send it to all
- * the partitions of the input data(RDD).
- * 2. For each partition, create the data writer, and write the data of the partition with this
- * writer. If all the data are written successfully, call {@link DataWriter#commit()}. If
- * exception happens during the writing, call {@link DataWriter#abort()}.
- * 3. If all writers are successfully committed, call {@link #commit(WriterCommitMessage[])}. If
- * some writers are aborted, or the job failed with an unknown reason, call
- * {@link #abort(WriterCommitMessage[])}.
- *
- * While Spark will retry failed writing tasks, Spark won't retry failed writing jobs. Users should
- * do it manually in their Spark applications if they want to retry.
- *
- * Please refer to the documentation of commit/abort methods for detailed specifications.
- */
-@InterfaceStability.Evolving
-public interface BatchWriteSupport {
-
- /**
- * Creates a writer factory which will be serialized and sent to executors.
- *
- * If this method fails (by throwing an exception), the action will fail and no Spark job will be
- * submitted.
- */
- DataWriterFactory createBatchWriterFactory();
-
- /**
- * Returns whether Spark should use the commit coordinator to ensure that at most one task for
- * each partition commits.
- *
- * @return true if commit coordinator should be used, false otherwise.
- */
- default boolean useCommitCoordinator() {
- return true;
- }
-
- /**
- * Handles a commit message on receiving from a successful data writer.
- *
- * If this method fails (by throwing an exception), this writing job is considered to to have been
- * failed, and {@link #abort(WriterCommitMessage[])} would be called.
- */
- default void onDataWriterCommit(WriterCommitMessage message) {}
-
- /**
- * Commits this writing job 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 to have been
- * failed, and {@link #abort(WriterCommitMessage[])} would be called. The state of the destination
- * is undefined and @{@link #abort(WriterCommitMessage[])} may not be able to deal with it.
- *
- * Note that speculative execution may cause multiple tasks to run for a partition. By default,
- * Spark uses the commit coordinator to allow at most one task to commit. Implementations can
- * disable this behavior by overriding {@link #useCommitCoordinator()}. If disabled, multiple
- * tasks may have committed successfully and one successful commit message per task will be
- * passed to this commit method. The remaining commit messages are ignored by Spark.
- */
- void commit(WriterCommitMessage[] messages);
-
- /**
- * Aborts this writing job because some data writers are failed and keep failing when retry,
- * or the Spark job fails with some unknown reasons,
- * or {@link #onDataWriterCommit(WriterCommitMessage)} fails,
- * or {@link #commit(WriterCommitMessage[])} fails.
- *
- * If this method fails (by throwing an exception), the underlying data source may require manual
- * cleanup.
- *
- * Unless the abort is triggered by the failure of commit, the given messages should have some
- * null slots as there maybe only a few data writers that are committed before the abort
- * happens, or some data writers were committed but their commit messages haven't reached the
- * driver when the abort is triggered. So this is just a "best effort" for data sources to
- * clean up the data left by data writers.
- */
- void abort(WriterCommitMessage[] messages);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java
new file mode 100644
index 0000000..385fc29
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java
@@ -0,0 +1,116 @@
+/*
+ * 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;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.StreamWriteSupport;
+import org.apache.spark.sql.sources.v2.WriteSupport;
+import org.apache.spark.sql.streaming.OutputMode;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * A data source writer that is returned by
+ * {@link WriteSupport#createWriter(String, StructType, SaveMode, DataSourceOptions)}/
+ * {@link StreamWriteSupport#createStreamWriter(
+ * String, StructType, OutputMode, DataSourceOptions)}.
+ * It can mix in various writing optimization interfaces to speed up the data saving. The actual
+ * writing logic is delegated to {@link DataWriter}.
+ *
+ * If an exception was throw when applying any of these writing optimizations, the action will fail
+ * and no Spark job will be submitted.
+ *
+ * The writing procedure is:
+ * 1. Create a writer factory by {@link #createWriterFactory()}, serialize and send it to all the
+ * partitions of the input data(RDD).
+ * 2. For each partition, create the data writer, and write the data of the partition with this
+ * writer. If all the data are written successfully, call {@link DataWriter#commit()}. If
+ * exception happens during the writing, call {@link DataWriter#abort()}.
+ * 3. If all writers are successfully committed, call {@link #commit(WriterCommitMessage[])}. If
+ * some writers are aborted, or the job failed with an unknown reason, call
+ * {@link #abort(WriterCommitMessage[])}.
+ *
+ * While Spark will retry failed writing tasks, Spark won't retry failed writing jobs. Users should
+ * do it manually in their Spark applications if they want to retry.
+ *
+ * Please refer to the documentation of commit/abort methods for detailed specifications.
+ */
+@InterfaceStability.Evolving
+public interface DataSourceWriter {
+
+ /**
+ * Creates a writer factory which will be serialized and sent to executors.
+ *
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
+ */
+ DataWriterFactory<InternalRow> createWriterFactory();
+
+ /**
+ * Returns whether Spark should use the commit coordinator to ensure that at most one task for
+ * each partition commits.
+ *
+ * @return true if commit coordinator should be used, false otherwise.
+ */
+ default boolean useCommitCoordinator() {
+ return true;
+ }
+
+ /**
+ * Handles a commit message on receiving from a successful data writer.
+ *
+ * If this method fails (by throwing an exception), this writing job is considered to to have been
+ * failed, and {@link #abort(WriterCommitMessage[])} would be called.
+ */
+ default void onDataWriterCommit(WriterCommitMessage message) {}
+
+ /**
+ * Commits this writing job 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 to have been
+ * failed, and {@link #abort(WriterCommitMessage[])} would be called. The state of the destination
+ * is undefined and @{@link #abort(WriterCommitMessage[])} may not be able to deal with it.
+ *
+ * Note that speculative execution may cause multiple tasks to run for a partition. By default,
+ * Spark uses the commit coordinator to allow at most one task to commit. Implementations can
+ * disable this behavior by overriding {@link #useCommitCoordinator()}. If disabled, multiple
+ * tasks may have committed successfully and one successful commit message per task will be
+ * passed to this commit method. The remaining commit messages are ignored by Spark.
+ */
+ void commit(WriterCommitMessage[] messages);
+
+ /**
+ * Aborts this writing job because some data writers are failed and keep failing when retry,
+ * or the Spark job fails with some unknown reasons,
+ * or {@link #onDataWriterCommit(WriterCommitMessage)} fails,
+ * or {@link #commit(WriterCommitMessage[])} fails.
+ *
+ * If this method fails (by throwing an exception), the underlying data source may require manual
+ * cleanup.
+ *
+ * Unless the abort is triggered by the failure of commit, the given messages should have some
+ * null slots as there maybe only a few data writers that are committed before the abort
+ * happens, or some data writers were committed but their commit messages haven't reached the
+ * driver when the abort is triggered. So this is just a "best effort" for data sources to
+ * clean up the data left by data writers.
+ */
+ void abort(WriterCommitMessage[] messages);
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
index 5fb0679..27dc5ea 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java
@@ -22,7 +22,7 @@ import java.io.IOException;
import org.apache.spark.annotation.InterfaceStability;
/**
- * A data writer returned by {@link DataWriterFactory#createWriter(int, long)} and is
+ * A data writer returned by {@link DataWriterFactory#createDataWriter(int, long, long)} and is
* responsible for writing data for an input RDD partition.
*
* One Spark task has one exclusive data writer, so there is no thread-safe concern.
@@ -36,11 +36,11 @@ import org.apache.spark.annotation.InterfaceStability;
*
* If this data writer succeeds(all records are successfully written and {@link #commit()}
* succeeds), a {@link WriterCommitMessage} will be sent to the driver side and pass to
- * {@link BatchWriteSupport#commit(WriterCommitMessage[])} with commit messages from other data
+ * {@link DataSourceWriter#commit(WriterCommitMessage[])} with commit messages from other data
* writers. If this data writer fails(one record fails to write or {@link #commit()} fails), an
* exception will be sent to the driver side, and Spark may retry this writing task a few times.
- * In each retry, {@link DataWriterFactory#createWriter(int, long)} will receive a
- * different `taskId`. Spark will call {@link BatchWriteSupport#abort(WriterCommitMessage[])}
+ * In each retry, {@link DataWriterFactory#createDataWriter(int, long, long)} will receive a
+ * different `taskId`. Spark will call {@link DataSourceWriter#abort(WriterCommitMessage[])}
* when the configured number of retries is exhausted.
*
* Besides the retry mechanism, Spark may launch speculative tasks if the existing writing task
@@ -71,11 +71,11 @@ public interface DataWriter<T> {
/**
* Commits this writer after all records are written successfully, returns a commit message which
* will be sent back to driver side and passed to
- * {@link BatchWriteSupport#commit(WriterCommitMessage[])}.
+ * {@link DataSourceWriter#commit(WriterCommitMessage[])}.
*
* The written data should only be visible to data source readers after
- * {@link BatchWriteSupport#commit(WriterCommitMessage[])} succeeds, which means this method
- * should still "hide" the written data and ask the {@link BatchWriteSupport} at driver side to
+ * {@link DataSourceWriter#commit(WriterCommitMessage[])} succeeds, which means this method
+ * should still "hide" the written data and ask the {@link DataSourceWriter} at driver side to
* do the final commit via {@link WriterCommitMessage}.
*
* If this method fails (by throwing an exception), {@link #abort()} will be called and this
@@ -93,7 +93,7 @@ public interface DataWriter<T> {
* failed.
*
* If this method fails(by throwing an exception), the underlying data source may have garbage
- * that need to be cleaned by {@link BatchWriteSupport#abort(WriterCommitMessage[])} or manually,
+ * that need to be cleaned by {@link DataSourceWriter#abort(WriterCommitMessage[])} or manually,
* but these garbage should not be visible to data source readers.
*
* @throws IOException if failure happens during disk/network IO like writing files.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java
index 19a36dd..3d337b6 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java
@@ -19,20 +19,18 @@ package org.apache.spark.sql.sources.v2.writer;
import java.io.Serializable;
-import org.apache.spark.TaskContext;
import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.catalyst.InternalRow;
/**
- * A factory of {@link DataWriter} returned by {@link BatchWriteSupport#createBatchWriterFactory()},
+ * A factory of {@link DataWriter} returned by {@link DataSourceWriter#createWriterFactory()},
* which is responsible for creating and initializing the actual data writer at executor side.
*
* Note that, the writer factory will be serialized and sent to executors, then the data writer
- * will be created on executors and do the actual writing. So this interface must be
+ * will be created on executors and do the actual writing. So {@link DataWriterFactory} must be
* serializable and {@link DataWriter} doesn't need to be.
*/
@InterfaceStability.Evolving
-public interface DataWriterFactory extends Serializable {
+public interface DataWriterFactory<T> extends Serializable {
/**
* Returns a data writer to do the actual writing work. Note that, Spark will reuse the same data
@@ -40,16 +38,19 @@ public interface DataWriterFactory extends Serializable {
* are responsible for defensive copies if necessary, e.g. copy the data before buffer it in a
* list.
*
- * If this method fails (by throwing an exception), the corresponding Spark write task would fail
- * and get retried until hitting the maximum retry times.
+ * If this method fails (by throwing an exception), the action will fail and no Spark job will be
+ * submitted.
*
* @param partitionId A unique id of the RDD partition that the returned writer will process.
* Usually Spark processes many RDD partitions at the same time,
* implementations should use the partition id to distinguish writers for
* different partitions.
- * @param taskId The task id returned by {@link TaskContext#taskAttemptId()}. Spark may run
- * multiple tasks for the same partition (due to speculation or task failures,
- * for example).
+ * @param taskId A unique identifier for a task that is performing the write of the partition
+ * data. Spark may run multiple tasks for the same partition (due to speculation
+ * or task failures, for example).
+ * @param epochId A monotonically increasing id for streaming queries that are split in to
+ * discrete periods of execution. For non-streaming queries,
+ * this ID will always be 0.
*/
- DataWriter<InternalRow> createWriter(int partitionId, long taskId);
+ DataWriter<T> createDataWriter(int partitionId, long taskId, long epochId);
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java
index 123335c..9e38836 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java
@@ -19,16 +19,15 @@ package org.apache.spark.sql.sources.v2.writer;
import java.io.Serializable;
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport;
import org.apache.spark.annotation.InterfaceStability;
/**
* A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side
- * as the input parameter of {@link BatchWriteSupport#commit(WriterCommitMessage[])} or
- * {@link StreamingWriteSupport#commit(long, WriterCommitMessage[])}.
+ * as the input parameter of {@link DataSourceWriter#commit(WriterCommitMessage[])}.
*
- * This is an empty interface, data sources should define their own message class and use it when
- * generating messages at executor side and handling the messages at driver side.
+ * This is an empty interface, data sources should define their own message class and use it in
+ * their {@link DataWriter#commit()} and {@link DataSourceWriter#commit(WriterCommitMessage[])}
+ * implementations.
*/
@InterfaceStability.Evolving
public interface WriterCommitMessage extends Serializable {}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java
new file mode 100644
index 0000000..a316b2a
--- /dev/null
+++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java
@@ -0,0 +1,71 @@
+/*
+ * 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.streaming;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
+import org.apache.spark.sql.sources.v2.writer.DataWriter;
+import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
+
+/**
+ * A {@link DataSourceWriter} for use with structured streaming.
+ *
+ * Streaming queries are divided into intervals of data called epochs, with a monotonically
+ * increasing numeric ID. This writer handles commits and aborts for each successive epoch.
+ */
+@InterfaceStability.Evolving
+public interface StreamWriter extends DataSourceWriter {
+ /**
+ * 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[])}.
+ *
+ * The execution engine may call commit() multiple times for the same epoch in some circumstances.
+ * To support exactly-once data semantics, implementations must ensure that multiple commits for
+ * the same epoch are idempotent.
+ */
+ void commit(long epochId, WriterCommitMessage[] messages);
+
+ /**
+ * Aborts this writing job because some data writers are failed and keep failing when retried, or
+ * the Spark job fails with some unknown reasons, or {@link #commit(WriterCommitMessage[])} fails.
+ *
+ * If this method fails (by throwing an exception), the underlying data source may require manual
+ * cleanup.
+ *
+ * Unless the abort is triggered by the failure of commit, the given messages will have some
+ * null slots, as there may be only a few data writers that were committed before the abort
+ * happens, or some data writers were committed but their commit messages haven't reached the
+ * driver when the abort is triggered. So this is just a "best effort" for data sources to
+ * clean up the data left by data writers.
+ */
+ void abort(long epochId, WriterCommitMessage[] messages);
+
+ default void commit(WriterCommitMessage[] messages) {
+ throw new UnsupportedOperationException(
+ "Commit without epoch should not be called with StreamWriter");
+ }
+
+ default void abort(WriterCommitMessage[] messages) {
+ throw new UnsupportedOperationException(
+ "Abort without epoch should not be called with StreamWriter");
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java
deleted file mode 100644
index a4da24f..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.writer.streaming;
-
-import java.io.Serializable;
-
-import org.apache.spark.TaskContext;
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.sources.v2.writer.DataWriter;
-
-/**
- * A factory of {@link DataWriter} returned by
- * {@link StreamingWriteSupport#createStreamingWriterFactory()}, which is responsible for creating
- * and initializing the actual data writer at executor side.
- *
- * Note that, the writer factory will be serialized and sent to executors, then the data writer
- * will be created on executors and do the actual writing. So this interface must be
- * serializable and {@link DataWriter} doesn't need to be.
- */
-@InterfaceStability.Evolving
-public interface StreamingDataWriterFactory extends Serializable {
-
- /**
- * Returns a data writer to do the actual writing work. Note that, Spark will reuse the same data
- * object instance when sending data to the data writer, for better performance. Data writers
- * are responsible for defensive copies if necessary, e.g. copy the data before buffer it in a
- * list.
- *
- * If this method fails (by throwing an exception), the corresponding Spark write task would fail
- * and get retried until hitting the maximum retry times.
- *
- * @param partitionId A unique id of the RDD partition that the returned writer will process.
- * Usually Spark processes many RDD partitions at the same time,
- * implementations should use the partition id to distinguish writers for
- * different partitions.
- * @param taskId The task id returned by {@link TaskContext#taskAttemptId()}. Spark may run
- * multiple tasks for the same partition (due to speculation or task failures,
- * for example).
- * @param epochId A monotonically increasing id for streaming queries that are split in to
- * discrete periods of execution.
- */
- DataWriter<InternalRow> createWriter(int partitionId, long taskId, long epochId);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java
deleted file mode 100644
index 3fdfac5..0000000
--- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.sources.v2.writer.streaming;
-
-import org.apache.spark.annotation.InterfaceStability;
-import org.apache.spark.sql.sources.v2.writer.DataWriter;
-import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
-
-/**
- * An interface that defines how to write the data to data source for streaming processing.
- *
- * Streaming queries are divided into intervals of data called epochs, with a monotonically
- * increasing numeric ID. This writer handles commits and aborts for each successive epoch.
- */
-@InterfaceStability.Evolving
-public interface StreamingWriteSupport {
-
- /**
- * Creates a writer factory which will be serialized and sent to executors.
- *
- * If this method fails (by throwing an exception), the action will fail and no Spark job will be
- * submitted.
- */
- StreamingDataWriterFactory createStreamingWriterFactory();
-
- /**
- * 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(long, WriterCommitMessage[])}.
- *
- * The execution engine may call `commit` multiple times for the same epoch in some circumstances.
- * To support exactly-once data semantics, implementations must ensure that multiple commits for
- * the same epoch are idempotent.
- */
- void commit(long epochId, WriterCommitMessage[] messages);
-
- /**
- * Aborts this writing job because some data writers are failed and keep failing when retried, or
- * the Spark job fails with some unknown reasons, or {@link #commit(long, WriterCommitMessage[])}
- * fails.
- *
- * If this method fails (by throwing an exception), the underlying data source may require manual
- * cleanup.
- *
- * Unless the abort is triggered by the failure of commit, the given messages will have some
- * null slots, as there may be only a few data writers that were committed before the abort
- * happens, or some data writers were committed but their commit messages haven't reached the
- * driver when the abort is triggered. So this is just a "best effort" for data sources to
- * clean up the data left by data writers.
- */
- void abort(long epochId, WriterCommitMessage[] messages);
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index e6c2cba..371ec70 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, DataSourceOptions, DataSourceV2}
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport}
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.unsafe.types.UTF8String
@@ -194,7 +194,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging {
val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf)
if (classOf[DataSourceV2].isAssignableFrom(cls)) {
val ds = cls.newInstance().asInstanceOf[DataSourceV2]
- if (ds.isInstanceOf[BatchReadSupportProvider]) {
+ if (ds.isInstanceOf[ReadSupport]) {
val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
ds = ds, conf = sparkSession.sessionState.conf)
val pathsOption = {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index dfb8c47..4aeddfd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -240,7 +240,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
if (classOf[DataSourceV2].isAssignableFrom(cls)) {
val source = cls.newInstance().asInstanceOf[DataSourceV2]
source match {
- case provider: BatchWriteSupportProvider =>
+ case ws: WriteSupport =>
val options = extraOptions ++
DataSourceV2Utils.extractSessionConfigs(source, df.sparkSession.sessionState.conf)
@@ -251,10 +251,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
}
} else {
- val writer = provider.createBatchWriteSupport(
- UUID.randomUUID().toString,
- df.logicalPlan.output.toStructType,
- mode,
+ val writer = ws.createWriter(
+ UUID.randomUUID.toString, df.logicalPlan.output.toStructType, mode,
new DataSourceOptions(options.asJava))
if (writer.isPresent) {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
index f62f734..7828298 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala
@@ -17,22 +17,19 @@
package org.apache.spark.sql.execution.datasources.v2
-import org.apache.spark._
+import scala.reflect.ClassTag
+
+import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext}
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory}
+import org.apache.spark.sql.sources.v2.reader.InputPartition
-class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition)
+class DataSourceRDDPartition[T : ClassTag](val index: Int, val inputPartition: InputPartition[T])
extends Partition with Serializable
-// TODO: we should have 2 RDDs: an RDD[InternalRow] for row-based scan, an `RDD[ColumnarBatch]` for
-// columnar scan.
-class DataSourceRDD(
+class DataSourceRDD[T: ClassTag](
sc: SparkContext,
- @transient private val inputPartitions: Seq[InputPartition],
- partitionReaderFactory: PartitionReaderFactory,
- columnarReads: Boolean)
- extends RDD[InternalRow](sc, Nil) {
+ @transient private val inputPartitions: Seq[InputPartition[T]])
+ extends RDD[T](sc, Nil) {
override protected def getPartitions: Array[Partition] = {
inputPartitions.zipWithIndex.map {
@@ -40,21 +37,11 @@ class DataSourceRDD(
}.toArray
}
- private def castPartition(split: Partition): DataSourceRDDPartition = split match {
- case p: DataSourceRDDPartition => p
- case _ => throw new SparkException(s"[BUG] Not a DataSourceRDDPartition: $split")
- }
-
- override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
- val inputPartition = castPartition(split).inputPartition
- val reader: PartitionReader[_] = if (columnarReads) {
- partitionReaderFactory.createColumnarReader(inputPartition)
- } else {
- partitionReaderFactory.createReader(inputPartition)
- }
-
+ override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+ val reader = split.asInstanceOf[DataSourceRDDPartition[T]].inputPartition
+ .createPartitionReader()
context.addTaskCompletionListener[Unit](_ => reader.close())
- val iter = new Iterator[Any] {
+ val iter = new Iterator[T] {
private[this] var valuePrepared = false
override def hasNext: Boolean = {
@@ -64,7 +51,7 @@ class DataSourceRDD(
valuePrepared
}
- override def next(): Any = {
+ override def next(): T = {
if (!hasNext) {
throw new java.util.NoSuchElementException("End of stream")
}
@@ -72,11 +59,10 @@ class DataSourceRDD(
reader.get()
}
}
- // TODO: SPARK-25083 remove the type erasure hack in data source scan
- new InterruptibleIterator(context, iter.asInstanceOf[Iterator[InternalRow]])
+ new InterruptibleIterator(context, iter)
}
override def getPreferredLocations(split: Partition): Seq[String] = {
- castPartition(split).inputPartition.preferredLocations()
+ split.asInstanceOf[DataSourceRDDPartition[T]].inputPartition.preferredLocations()
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
index f7e2959..abc5fb9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
@@ -27,21 +27,21 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.sources.DataSourceRegister
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, BatchWriteSupportProvider, DataSourceOptions, DataSourceV2}
-import org.apache.spark.sql.sources.v2.reader.{BatchReadSupport, ReadSupport, ScanConfigBuilder, SupportsReportStatistics}
-import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, WriteSupport}
+import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsReportStatistics}
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter
import org.apache.spark.sql.types.StructType
/**
* A logical plan representing a data source v2 scan.
*
* @param source An instance of a [[DataSourceV2]] implementation.
- * @param options The options for this scan. Used to create fresh [[BatchWriteSupport]].
- * @param userSpecifiedSchema The user-specified schema for this scan.
+ * @param options The options for this scan. Used to create fresh [[DataSourceReader]].
+ * @param userSpecifiedSchema The user-specified schema for this scan. Used to create fresh
+ * [[DataSourceReader]].
*/
case class DataSourceV2Relation(
source: DataSourceV2,
- readSupport: BatchReadSupport,
output: Seq[AttributeReference],
options: Map[String, String],
tableIdent: Option[TableIdentifier] = None,
@@ -58,12 +58,13 @@ case class DataSourceV2Relation(
override def simpleString: String = "RelationV2 " + metadataString
- def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema)
+ def newReader(): DataSourceReader = source.createReader(options, userSpecifiedSchema)
- override def computeStats(): Statistics = readSupport match {
+ def newWriter(): DataSourceWriter = source.createWriter(options, schema)
+
+ override def computeStats(): Statistics = newReader match {
case r: SupportsReportStatistics =>
- val statistics = r.estimateStatistics(readSupport.newScanConfigBuilder().build())
- Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
+ Statistics(sizeInBytes = r.estimateStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
case _ =>
Statistics(sizeInBytes = conf.defaultSizeInBytes)
}
@@ -84,8 +85,7 @@ case class StreamingDataSourceV2Relation(
output: Seq[AttributeReference],
source: DataSourceV2,
options: Map[String, String],
- readSupport: ReadSupport,
- scanConfigBuilder: ScanConfigBuilder)
+ reader: DataSourceReader)
extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat {
override def isStreaming: Boolean = true
@@ -99,8 +99,7 @@ case class StreamingDataSourceV2Relation(
// TODO: unify the equal/hashCode implementation for all data source v2 query plans.
override def equals(other: Any): Boolean = other match {
case other: StreamingDataSourceV2Relation =>
- output == other.output && readSupport.getClass == other.readSupport.getClass &&
- options == other.options
+ output == other.output && reader.getClass == other.reader.getClass && options == other.options
case _ => false
}
@@ -108,10 +107,9 @@ case class StreamingDataSourceV2Relation(
Seq(output, source, options).hashCode()
}
- override def computeStats(): Statistics = readSupport match {
+ override def computeStats(): Statistics = reader match {
case r: SupportsReportStatistics =>
- val statistics = r.estimateStatistics(scanConfigBuilder.build())
- Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
+ Statistics(sizeInBytes = r.estimateStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes))
case _ =>
Statistics(sizeInBytes = conf.defaultSizeInBytes)
}
@@ -119,19 +117,19 @@ case class StreamingDataSourceV2Relation(
object DataSourceV2Relation {
private implicit class SourceHelpers(source: DataSourceV2) {
- def asReadSupportProvider: BatchReadSupportProvider = {
+ def asReadSupport: ReadSupport = {
source match {
- case provider: BatchReadSupportProvider =>
- provider
+ case support: ReadSupport =>
+ support
case _ =>
throw new AnalysisException(s"Data source is not readable: $name")
}
}
- def asWriteSupportProvider: BatchWriteSupportProvider = {
+ def asWriteSupport: WriteSupport = {
source match {
- case provider: BatchWriteSupportProvider =>
- provider
+ case support: WriteSupport =>
+ support
case _ =>
throw new AnalysisException(s"Data source is not writable: $name")
}
@@ -146,26 +144,23 @@ object DataSourceV2Relation {
}
}
- def createReadSupport(
+ def createReader(
options: Map[String, String],
- userSpecifiedSchema: Option[StructType]): BatchReadSupport = {
+ userSpecifiedSchema: Option[StructType]): DataSourceReader = {
val v2Options = new DataSourceOptions(options.asJava)
userSpecifiedSchema match {
case Some(s) =>
- asReadSupportProvider.createBatchReadSupport(s, v2Options)
+ asReadSupport.createReader(s, v2Options)
case _ =>
- asReadSupportProvider.createBatchReadSupport(v2Options)
+ asReadSupport.createReader(v2Options)
}
}
- def createWriteSupport(
+ def createWriter(
options: Map[String, String],
- schema: StructType): BatchWriteSupport = {
- asWriteSupportProvider.createBatchWriteSupport(
- UUID.randomUUID().toString,
- schema,
- SaveMode.Append,
- new DataSourceOptions(options.asJava)).get
+ schema: StructType): DataSourceWriter = {
+ val v2Options = new DataSourceOptions(options.asJava)
+ asWriteSupport.createWriter(UUID.randomUUID.toString, schema, SaveMode.Append, v2Options).get
}
}
@@ -174,16 +169,15 @@ object DataSourceV2Relation {
options: Map[String, String],
tableIdent: Option[TableIdentifier] = None,
userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = {
- val readSupport = source.createReadSupport(options, userSpecifiedSchema)
- val output = readSupport.fullSchema().toAttributes
+ val reader = source.createReader(options, userSpecifiedSchema)
val ident = tableIdent.orElse(tableFromOptions(options))
DataSourceV2Relation(
- source, readSupport, output, options, ident, userSpecifiedSchema)
+ source, reader.readSchema().toAttributes, options, ident, userSpecifiedSchema)
}
private def tableFromOptions(options: Map[String, String]): Option[TableIdentifier] = {
options
- .get(DataSourceOptions.TABLE_KEY)
- .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY)))
+ .get(DataSourceOptions.TABLE_KEY)
+ .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY)))
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
index 04a9773..c8494f9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.execution.datasources.v2
+import scala.collection.JavaConverters._
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
@@ -26,7 +28,8 @@ import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeSta
import org.apache.spark.sql.execution.streaming.continuous._
import org.apache.spark.sql.sources.v2.DataSourceV2
import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousReadSupport, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader
+import org.apache.spark.sql.vectorized.ColumnarBatch
/**
* Physical plan node for scanning data from a data source.
@@ -36,8 +39,7 @@ case class DataSourceV2ScanExec(
@transient source: DataSourceV2,
@transient options: Map[String, String],
@transient pushedFilters: Seq[Expression],
- @transient readSupport: ReadSupport,
- @transient scanConfig: ScanConfig)
+ @transient reader: DataSourceReader)
extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan {
override def simpleString: String = "ScanV2 " + metadataString
@@ -45,8 +47,7 @@ case class DataSourceV2ScanExec(
// TODO: unify the equal/hashCode implementation for all data source v2 query plans.
override def equals(other: Any): Boolean = other match {
case other: DataSourceV2ScanExec =>
- output == other.output && readSupport.getClass == other.readSupport.getClass &&
- options == other.options
+ output == other.output && reader.getClass == other.reader.getClass && options == other.options
case _ => false
}
@@ -54,39 +55,36 @@ case class DataSourceV2ScanExec(
Seq(output, source, options).hashCode()
}
- override def outputPartitioning: physical.Partitioning = readSupport match {
- case _ if partitions.length == 1 =>
+ override def outputPartitioning: physical.Partitioning = reader match {
+ case r: SupportsScanColumnarBatch if r.enableBatchRead() && batchPartitions.size == 1 =>
+ SinglePartition
+
+ case r: SupportsScanColumnarBatch if !r.enableBatchRead() && partitions.size == 1 =>
+ SinglePartition
+
+ case r if !r.isInstanceOf[SupportsScanColumnarBatch] && partitions.size == 1 =>
SinglePartition
case s: SupportsReportPartitioning =>
new DataSourcePartitioning(
- s.outputPartitioning(scanConfig), AttributeMap(output.map(a => a -> a.name)))
+ s.outputPartitioning(), AttributeMap(output.map(a => a -> a.name)))
case _ => super.outputPartitioning
}
- private lazy val partitions: Seq[InputPartition] = readSupport.planInputPartitions(scanConfig)
-
- private lazy val readerFactory = readSupport match {
- case r: BatchReadSupport => r.createReaderFactory(scanConfig)
- case r: MicroBatchReadSupport => r.createReaderFactory(scanConfig)
- case r: ContinuousReadSupport => r.createContinuousReaderFactory(scanConfig)
- case _ => throw new IllegalStateException("unknown read support: " + readSupport)
+ private lazy val partitions: Seq[InputPartition[InternalRow]] = {
+ reader.planInputPartitions().asScala
}
- // TODO: clean this up when we have dedicated scan plan for continuous streaming.
- override val supportsBatch: Boolean = {
- require(partitions.forall(readerFactory.supportColumnarReads) ||
- !partitions.exists(readerFactory.supportColumnarReads),
- "Cannot mix row-based and columnar input partitions.")
-
- partitions.exists(readerFactory.supportColumnarReads)
+ private lazy val batchPartitions: Seq[InputPartition[ColumnarBatch]] = reader match {
+ case r: SupportsScanColumnarBatch if r.enableBatchRead() =>
+ assert(!reader.isInstanceOf[ContinuousReader],
+ "continuous stream reader does not support columnar read yet.")
+ r.planBatchInputPartitions().asScala
}
- private lazy val inputRDD: RDD[InternalRow] = readSupport match {
- case _: ContinuousReadSupport =>
- assert(!supportsBatch,
- "continuous stream reader does not support columnar read yet.")
+ private lazy val inputRDD: RDD[InternalRow] = reader match {
+ case _: ContinuousReader =>
EpochCoordinatorRef.get(
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
sparkContext.env)
@@ -95,17 +93,22 @@ case class DataSourceV2ScanExec(
sparkContext,
sqlContext.conf.continuousStreamingExecutorQueueSize,
sqlContext.conf.continuousStreamingExecutorPollIntervalMs,
- partitions,
- schema,
- readerFactory.asInstanceOf[ContinuousPartitionReaderFactory])
+ partitions).asInstanceOf[RDD[InternalRow]]
+
+ case r: SupportsScanColumnarBatch if r.enableBatchRead() =>
+ new DataSourceRDD(sparkContext, batchPartitions).asInstanceOf[RDD[InternalRow]]
case _ =>
- new DataSourceRDD(
- sparkContext, partitions, readerFactory.asInstanceOf[PartitionReaderFactory], supportsBatch)
+ new DataSourceRDD(sparkContext, partitions).asInstanceOf[RDD[InternalRow]]
}
override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD)
+ override val supportsBatch: Boolean = reader match {
+ case r: SupportsScanColumnarBatch if r.enableBatchRead() => true
+ case _ => false
+ }
+
override protected def needsUnsafeRowConversion: Boolean = false
override protected def doExecute(): RDD[InternalRow] = {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 9a3109e..9d97d3b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, Rep
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport
+import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader
object DataSourceV2Strategy extends Strategy {
@@ -37,9 +37,9 @@ object DataSourceV2Strategy extends Strategy {
* @return pushed filter and post-scan filters.
*/
private def pushFilters(
- configBuilder: ScanConfigBuilder,
+ reader: DataSourceReader,
filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = {
- configBuilder match {
+ reader match {
case r: SupportsPushDownFilters =>
// A map from translated data source filters to original catalyst filter expressions.
val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression]
@@ -71,43 +71,41 @@ object DataSourceV2Strategy extends Strategy {
/**
* Applies column pruning to the data source, w.r.t. the references of the given expressions.
*
- * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown),
- * and new output attributes after column pruning.
+ * @return new output attributes after column pruning.
*/
// TODO: nested column pruning.
private def pruneColumns(
- configBuilder: ScanConfigBuilder,
+ reader: DataSourceReader,
relation: DataSourceV2Relation,
- exprs: Seq[Expression]): (ScanConfig, Seq[AttributeReference]) = {
- configBuilder match {
+ exprs: Seq[Expression]): Seq[AttributeReference] = {
+ reader match {
case r: SupportsPushDownRequiredColumns =>
val requiredColumns = AttributeSet(exprs.flatMap(_.references))
val neededOutput = relation.output.filter(requiredColumns.contains)
if (neededOutput != relation.output) {
r.pruneColumns(neededOutput.toStructType)
- val config = r.build()
val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap
- config -> config.readSchema().toAttributes.map {
+ r.readSchema().toAttributes.map {
// We have to keep the attribute id during transformation.
a => a.withExprId(nameToAttr(a.name).exprId)
}
} else {
- r.build() -> relation.output
+ relation.output
}
- case _ => configBuilder.build() -> relation.output
+ case _ => relation.output
}
}
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case PhysicalOperation(project, filters, relation: DataSourceV2Relation) =>
- val configBuilder = relation.readSupport.newScanConfigBuilder()
+ val reader = relation.newReader()
// `pushedFilters` will be pushed down and evaluated in the underlying data sources.
// `postScanFilters` need to be evaluated after the scan.
// `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter.
- val (pushedFilters, postScanFilters) = pushFilters(configBuilder, filters)
- val (config, output) = pruneColumns(configBuilder, relation, project ++ postScanFilters)
+ val (pushedFilters, postScanFilters) = pushFilters(reader, filters)
+ val output = pruneColumns(reader, relation, project ++ postScanFilters)
logInfo(
s"""
|Pushing operators to ${relation.source.getClass}
@@ -117,12 +115,7 @@ object DataSourceV2Strategy extends Strategy {
""".stripMargin)
val scan = DataSourceV2ScanExec(
- output,
- relation.source,
- relation.options,
- pushedFilters,
- relation.readSupport,
- config)
+ output, relation.source, relation.options, pushedFilters, reader)
val filterCondition = postScanFilters.reduceLeftOption(And)
val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)
@@ -131,26 +124,22 @@ object DataSourceV2Strategy extends Strategy {
ProjectExec(project, withFilter) :: Nil
case r: StreamingDataSourceV2Relation =>
- // TODO: support operator pushdown for streaming data sources.
- val scanConfig = r.scanConfigBuilder.build()
// ensure there is a projection, which will produce unsafe rows required by some operators
ProjectExec(r.output,
- DataSourceV2ScanExec(
- r.output, r.source, r.options, r.pushedFilters, r.readSupport, scanConfig)) :: Nil
+ DataSourceV2ScanExec(r.output, r.source, r.options, r.pushedFilters, r.reader)) :: Nil
case WriteToDataSourceV2(writer, query) =>
WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil
case AppendData(r: DataSourceV2Relation, query, _) =>
- WriteToDataSourceV2Exec(r.newWriteSupport(), planLater(query)) :: Nil
+ WriteToDataSourceV2Exec(r.newWriter(), planLater(query)) :: Nil
case WriteToContinuousDataSource(writer, query) =>
WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil
case Repartition(1, false, child) =>
- val isContinuous = child.find {
- case s: StreamingDataSourceV2Relation => s.readSupport.isInstanceOf[ContinuousReadSupport]
- case _ => false
+ val isContinuous = child.collectFirst {
+ case StreamingDataSourceV2Relation(_, _, _, r: ContinuousReader) => r
}.isDefined
if (isContinuous) {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala
index e9cc399..5267f5f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala
@@ -21,7 +21,6 @@ import java.util.regex.Pattern
import org.apache.spark.internal.Logging
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.sources.v2.{DataSourceV2, SessionConfigSupport}
private[sql] object DataSourceV2Utils extends Logging {
@@ -56,12 +55,4 @@ private[sql] object DataSourceV2Utils extends Logging {
case _ => Map.empty
}
-
- def failForUserSpecifiedSchema[T](ds: DataSourceV2): T = {
- val name = ds match {
- case register: DataSourceRegister => register.shortName()
- case _ => ds.getClass.getName
- }
- throw new UnsupportedOperationException(name + " source does not support user-specified schema")
- }
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
index c3f7b69..59ebb9b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
@@ -23,11 +23,15 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext}
import org.apache.spark.executor.CommitDeniedException
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.streaming.MicroBatchExecution
import org.apache.spark.sql.sources.v2.writer._
+import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
/**
@@ -35,8 +39,7 @@ import org.apache.spark.util.Utils
* specific logical plans, like [[org.apache.spark.sql.catalyst.plans.logical.AppendData]].
*/
@deprecated("Use specific logical plans like AppendData instead", "2.4.0")
-case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPlan)
- extends LogicalPlan {
+case class WriteToDataSourceV2(writer: DataSourceWriter, query: LogicalPlan) extends LogicalPlan {
override def children: Seq[LogicalPlan] = Seq(query)
override def output: Seq[Attribute] = Nil
}
@@ -44,48 +47,46 @@ case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPl
/**
* The physical plan for writing data into data source v2.
*/
-case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: SparkPlan)
- extends SparkPlan {
-
+case class WriteToDataSourceV2Exec(writer: DataSourceWriter, query: SparkPlan) extends SparkPlan {
override def children: Seq[SparkPlan] = Seq(query)
override def output: Seq[Attribute] = Nil
override protected def doExecute(): RDD[InternalRow] = {
- val writerFactory = writeSupport.createBatchWriterFactory()
- val useCommitCoordinator = writeSupport.useCommitCoordinator
+ val writeTask = writer.createWriterFactory()
+ val useCommitCoordinator = writer.useCommitCoordinator
val rdd = query.execute()
val messages = new Array[WriterCommitMessage](rdd.partitions.length)
- logInfo(s"Start processing data source write support: $writeSupport. " +
+ logInfo(s"Start processing data source writer: $writer. " +
s"The input RDD has ${messages.length} partitions.")
try {
sparkContext.runJob(
rdd,
(context: TaskContext, iter: Iterator[InternalRow]) =>
- DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator),
+ DataWritingSparkTask.run(writeTask, context, iter, useCommitCoordinator),
rdd.partitions.indices,
(index, message: WriterCommitMessage) => {
messages(index) = message
- writeSupport.onDataWriterCommit(message)
+ writer.onDataWriterCommit(message)
}
)
- logInfo(s"Data source write support $writeSupport is committing.")
- writeSupport.commit(messages)
- logInfo(s"Data source write support $writeSupport committed.")
+ logInfo(s"Data source writer $writer is committing.")
+ writer.commit(messages)
+ logInfo(s"Data source writer $writer committed.")
} catch {
case cause: Throwable =>
- logError(s"Data source write support $writeSupport is aborting.")
+ logError(s"Data source writer $writer is aborting.")
try {
- writeSupport.abort(messages)
+ writer.abort(messages)
} catch {
case t: Throwable =>
- logError(s"Data source write support $writeSupport failed to abort.")
+ logError(s"Data source writer $writer failed to abort.")
cause.addSuppressed(t)
throw new SparkException("Writing job failed.", cause)
}
- logError(s"Data source write support $writeSupport aborted.")
+ logError(s"Data source writer $writer aborted.")
cause match {
// Only wrap non fatal exceptions.
case NonFatal(e) => throw new SparkException("Writing job aborted.", e)
@@ -99,7 +100,7 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark
object DataWritingSparkTask extends Logging {
def run(
- writerFactory: DataWriterFactory,
+ writeTask: DataWriterFactory[InternalRow],
context: TaskContext,
iter: Iterator[InternalRow],
useCommitCoordinator: Boolean): WriterCommitMessage = {
@@ -108,7 +109,8 @@ object DataWritingSparkTask extends Logging {
val partId = context.partitionId()
val taskId = context.taskAttemptId()
val attemptId = context.attemptNumber()
- val dataWriter = writerFactory.createWriter(partId, taskId)
+ val epochId = Option(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY)).getOrElse("0")
+ val dataWriter = writeTask.createDataWriter(partId, taskId, epochId.toLong)
// write the data and commit this writer.
Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[3/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala
deleted file mode 100644
index 9f88416..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage}
-import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport}
-
-/**
- * A [[BatchWriteSupport]] used to hook V2 stream writers into a microbatch plan. It implements
- * the non-streaming interface, forwarding the epoch ID determined at construction to a wrapped
- * streaming write support.
- */
-class MicroBatchWritSupport(eppchId: Long, val writeSupport: StreamingWriteSupport)
- extends BatchWriteSupport {
-
- override def commit(messages: Array[WriterCommitMessage]): Unit = {
- writeSupport.commit(eppchId, messages)
- }
-
- override def abort(messages: Array[WriterCommitMessage]): Unit = {
- writeSupport.abort(eppchId, messages)
- }
-
- override def createBatchWriterFactory(): DataWriterFactory = {
- new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory())
- }
-}
-
-class MicroBatchWriterFactory(epochId: Long, streamingWriterFactory: StreamingDataWriterFactory)
- extends DataWriterFactory {
-
- override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = {
- streamingWriterFactory.createWriter(partitionId, taskId, epochId)
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala
new file mode 100644
index 0000000..2d43a7b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.sources
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.sources.v2.writer.{DataSourceWriter, DataWriterFactory, WriterCommitMessage}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
+
+/**
+ * A [[DataSourceWriter]] used to hook V2 stream writers into a microbatch plan. It implements
+ * the non-streaming interface, forwarding the batch ID determined at construction to a wrapped
+ * streaming writer.
+ */
+class MicroBatchWriter(batchId: Long, val writer: StreamWriter) extends DataSourceWriter {
+ override def commit(messages: Array[WriterCommitMessage]): Unit = {
+ writer.commit(batchId, messages)
+ }
+
+ override def abort(messages: Array[WriterCommitMessage]): Unit = writer.abort(batchId, messages)
+
+ override def createWriterFactory(): DataWriterFactory[InternalRow] = writer.createWriterFactory()
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala
index ac3c71c..f26e11d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala
@@ -21,18 +21,17 @@ import scala.collection.mutable
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage}
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory
+import org.apache.spark.sql.sources.v2.writer.{DataSourceWriter, DataWriter, DataWriterFactory, WriterCommitMessage}
/**
* A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery
- * to a [[BatchWriteSupport]] on the driver.
+ * to a [[DataSourceWriter]] on the driver.
*
* Note that, because it sends all rows to the driver, this factory will generally be unsuitable
* for production-quality sinks. It's intended for use in tests.
*/
-case object PackedRowWriterFactory extends StreamingDataWriterFactory {
- override def createWriter(
+case object PackedRowWriterFactory extends DataWriterFactory[InternalRow] {
+ override def createDataWriter(
partitionId: Int,
taskId: Long,
epochId: Long): DataWriter[InternalRow] = {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala
deleted file mode 100644
index 90680ea..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset}
-
-// A special `MicroBatchReadSupport` that can get latestOffset with a start offset.
-trait RateControlMicroBatchReadSupport extends MicroBatchReadSupport {
-
- override def latestOffset(): Offset = {
- throw new IllegalAccessException(
- "latestOffset should not be called for RateControlMicroBatchReadSupport")
- }
-
- def latestOffset(start: Offset): Offset
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala
deleted file mode 100644
index f536404..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import java.io._
-import java.nio.charset.StandardCharsets
-import java.util.concurrent.TimeUnit
-
-import org.apache.commons.io.IOUtils
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.network.util.JavaUtils
-import org.apache.spark.sql.SparkSession
-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.sources.v2.DataSourceOptions
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset}
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.util.{ManualClock, SystemClock}
-
-class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLocation: String)
- extends MicroBatchReadSupport with Logging {
- import RateStreamProvider._
-
- private[sources] val clock = {
- // The option to use a manual clock is provided only for unit testing purposes.
- if (options.getBoolean("useManualClock", false)) new ManualClock else new SystemClock
- }
-
- private val rowsPerSecond =
- options.get(ROWS_PER_SECOND).orElse("1").toLong
-
- private val rampUpTimeSeconds =
- Option(options.get(RAMP_UP_TIME).orElse(null.asInstanceOf[String]))
- .map(JavaUtils.timeStringAsSec(_))
- .getOrElse(0L)
-
- private val maxSeconds = Long.MaxValue / rowsPerSecond
-
- if (rampUpTimeSeconds > maxSeconds) {
- throw new ArithmeticException(
- s"Integer overflow. Max offset with $rowsPerSecond rowsPerSecond" +
- s" is $maxSeconds, but 'rampUpTimeSeconds' is $rampUpTimeSeconds.")
- }
-
- private[sources] val creationTimeMs = {
- val session = SparkSession.getActiveSession.orElse(SparkSession.getDefaultSession)
- require(session.isDefined)
-
- val metadataLog =
- new HDFSMetadataLog[LongOffset](session.get, checkpointLocation) {
- override def serialize(metadata: LongOffset, out: OutputStream): Unit = {
- val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
- writer.write("v" + VERSION + "\n")
- writer.write(metadata.json)
- writer.flush
- }
-
- override def deserialize(in: InputStream): LongOffset = {
- val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
- // HDFSMetadataLog guarantees that it never creates a partial file.
- assert(content.length != 0)
- if (content(0) == 'v') {
- val indexOfNewLine = content.indexOf("\n")
- if (indexOfNewLine > 0) {
- parseVersion(content.substring(0, indexOfNewLine), VERSION)
- LongOffset(SerializedOffset(content.substring(indexOfNewLine + 1)))
- } else {
- throw new IllegalStateException(
- s"Log file was malformed: failed to detect the log file version line.")
- }
- } else {
- throw new IllegalStateException(
- s"Log file was malformed: failed to detect the log file version line.")
- }
- }
- }
-
- metadataLog.get(0).getOrElse {
- val offset = LongOffset(clock.getTimeMillis())
- metadataLog.add(0, offset)
- logInfo(s"Start time: $offset")
- offset
- }.offset
- }
-
- @volatile private var lastTimeMs: Long = creationTimeMs
-
- override def initialOffset(): Offset = LongOffset(0L)
-
- override def latestOffset(): Offset = {
- val now = clock.getTimeMillis()
- if (lastTimeMs < now) {
- lastTimeMs = now
- }
- LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - creationTimeMs))
- }
-
- override def deserializeOffset(json: String): Offset = {
- LongOffset(json.toLong)
- }
-
- override def fullSchema(): StructType = SCHEMA
-
- override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
- }
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val sc = config.asInstanceOf[SimpleStreamingScanConfig]
- val startSeconds = sc.start.asInstanceOf[LongOffset].offset
- val endSeconds = sc.end.get.asInstanceOf[LongOffset].offset
- assert(startSeconds <= endSeconds, s"startSeconds($startSeconds) > endSeconds($endSeconds)")
- if (endSeconds > maxSeconds) {
- throw new ArithmeticException("Integer overflow. Max offset with " +
- s"$rowsPerSecond rowsPerSecond is $maxSeconds, but it's $endSeconds now.")
- }
- // Fix "lastTimeMs" for recovery
- if (lastTimeMs < TimeUnit.SECONDS.toMillis(endSeconds) + creationTimeMs) {
- lastTimeMs = TimeUnit.SECONDS.toMillis(endSeconds) + creationTimeMs
- }
- val rangeStart = valueAtSecond(startSeconds, rowsPerSecond, rampUpTimeSeconds)
- val rangeEnd = valueAtSecond(endSeconds, rowsPerSecond, rampUpTimeSeconds)
- logDebug(s"startSeconds: $startSeconds, endSeconds: $endSeconds, " +
- s"rangeStart: $rangeStart, rangeEnd: $rangeEnd")
-
- if (rangeStart == rangeEnd) {
- return Array.empty
- }
-
- val localStartTimeMs = creationTimeMs + TimeUnit.SECONDS.toMillis(startSeconds)
- val relativeMsPerValue =
- TimeUnit.SECONDS.toMillis(endSeconds - startSeconds).toDouble / (rangeEnd - rangeStart)
- val numPartitions = {
- val activeSession = SparkSession.getActiveSession
- require(activeSession.isDefined)
- Option(options.get(NUM_PARTITIONS).orElse(null.asInstanceOf[String]))
- .map(_.toInt)
- .getOrElse(activeSession.get.sparkContext.defaultParallelism)
- }
-
- (0 until numPartitions).map { p =>
- new RateStreamMicroBatchInputPartition(
- p, numPartitions, rangeStart, rangeEnd, localStartTimeMs, relativeMsPerValue)
- }.toArray
- }
-
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- RateStreamMicroBatchReaderFactory
- }
-
- override def commit(end: Offset): Unit = {}
-
- override def stop(): Unit = {}
-
- override def toString: String = s"RateStreamV2[rowsPerSecond=$rowsPerSecond, " +
- s"rampUpTimeSeconds=$rampUpTimeSeconds, " +
- s"numPartitions=${options.get(NUM_PARTITIONS).orElse("default")}"
-}
-
-case class RateStreamMicroBatchInputPartition(
- partitionId: Int,
- numPartitions: Int,
- rangeStart: Long,
- rangeEnd: Long,
- localStartTimeMs: Long,
- relativeMsPerValue: Double) extends InputPartition
-
-object RateStreamMicroBatchReaderFactory extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[RateStreamMicroBatchInputPartition]
- new RateStreamMicroBatchPartitionReader(p.partitionId, p.numPartitions, p.rangeStart,
- p.rangeEnd, p.localStartTimeMs, p.relativeMsPerValue)
- }
-}
-
-class RateStreamMicroBatchPartitionReader(
- partitionId: Int,
- numPartitions: Int,
- rangeStart: Long,
- rangeEnd: Long,
- localStartTimeMs: Long,
- relativeMsPerValue: Double) extends PartitionReader[InternalRow] {
- private var count: Long = 0
-
- override def next(): Boolean = {
- rangeStart + partitionId + numPartitions * count < rangeEnd
- }
-
- override def get(): InternalRow = {
- val currValue = rangeStart + partitionId + numPartitions * count
- count += 1
- val relative = math.round((currValue - rangeStart) * relativeMsPerValue)
- InternalRow(DateTimeUtils.fromMillis(relative + localStartTimeMs), currValue)
- }
-
- override def close(): Unit = {}
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
new file mode 100644
index 0000000..9e0d954
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala
@@ -0,0 +1,220 @@
+/*
+ * 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.sources
+
+import java.io._
+import java.nio.charset.StandardCharsets
+import java.util.Optional
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import org.apache.commons.io.IOUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.sql.{Row, SparkSession}
+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.sources.v2.DataSourceOptions
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.{ManualClock, SystemClock}
+
+class RateStreamMicroBatchReader(options: DataSourceOptions, checkpointLocation: String)
+ extends MicroBatchReader with Logging {
+ import RateStreamProvider._
+
+ private[sources] val clock = {
+ // The option to use a manual clock is provided only for unit testing purposes.
+ if (options.getBoolean("useManualClock", false)) new ManualClock else new SystemClock
+ }
+
+ private val rowsPerSecond =
+ options.get(ROWS_PER_SECOND).orElse("1").toLong
+
+ private val rampUpTimeSeconds =
+ Option(options.get(RAMP_UP_TIME).orElse(null.asInstanceOf[String]))
+ .map(JavaUtils.timeStringAsSec(_))
+ .getOrElse(0L)
+
+ private val maxSeconds = Long.MaxValue / rowsPerSecond
+
+ if (rampUpTimeSeconds > maxSeconds) {
+ throw new ArithmeticException(
+ s"Integer overflow. Max offset with $rowsPerSecond rowsPerSecond" +
+ s" is $maxSeconds, but 'rampUpTimeSeconds' is $rampUpTimeSeconds.")
+ }
+
+ private[sources] val creationTimeMs = {
+ val session = SparkSession.getActiveSession.orElse(SparkSession.getDefaultSession)
+ require(session.isDefined)
+
+ val metadataLog =
+ new HDFSMetadataLog[LongOffset](session.get, checkpointLocation) {
+ override def serialize(metadata: LongOffset, out: OutputStream): Unit = {
+ val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
+ writer.write("v" + VERSION + "\n")
+ writer.write(metadata.json)
+ writer.flush
+ }
+
+ override def deserialize(in: InputStream): LongOffset = {
+ val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
+ // HDFSMetadataLog guarantees that it never creates a partial file.
+ assert(content.length != 0)
+ if (content(0) == 'v') {
+ val indexOfNewLine = content.indexOf("\n")
+ if (indexOfNewLine > 0) {
+ parseVersion(content.substring(0, indexOfNewLine), VERSION)
+ LongOffset(SerializedOffset(content.substring(indexOfNewLine + 1)))
+ } else {
+ throw new IllegalStateException(
+ s"Log file was malformed: failed to detect the log file version line.")
+ }
+ } else {
+ throw new IllegalStateException(
+ s"Log file was malformed: failed to detect the log file version line.")
+ }
+ }
+ }
+
+ metadataLog.get(0).getOrElse {
+ val offset = LongOffset(clock.getTimeMillis())
+ metadataLog.add(0, offset)
+ logInfo(s"Start time: $offset")
+ offset
+ }.offset
+ }
+
+ @volatile private var lastTimeMs: Long = creationTimeMs
+
+ private var start: LongOffset = _
+ private var end: LongOffset = _
+
+ override def readSchema(): StructType = SCHEMA
+
+ override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = {
+ this.start = start.orElse(LongOffset(0L)).asInstanceOf[LongOffset]
+ this.end = end.orElse {
+ val now = clock.getTimeMillis()
+ if (lastTimeMs < now) {
+ lastTimeMs = now
+ }
+ LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - creationTimeMs))
+ }.asInstanceOf[LongOffset]
+ }
+
+ 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 deserializeOffset(json: String): Offset = {
+ LongOffset(json.toLong)
+ }
+
+ override def planInputPartitions(): java.util.List[InputPartition[InternalRow]] = {
+ val startSeconds = LongOffset.convert(start).map(_.offset).getOrElse(0L)
+ val endSeconds = LongOffset.convert(end).map(_.offset).getOrElse(0L)
+ assert(startSeconds <= endSeconds, s"startSeconds($startSeconds) > endSeconds($endSeconds)")
+ if (endSeconds > maxSeconds) {
+ throw new ArithmeticException("Integer overflow. Max offset with " +
+ s"$rowsPerSecond rowsPerSecond is $maxSeconds, but it's $endSeconds now.")
+ }
+ // Fix "lastTimeMs" for recovery
+ if (lastTimeMs < TimeUnit.SECONDS.toMillis(endSeconds) + creationTimeMs) {
+ lastTimeMs = TimeUnit.SECONDS.toMillis(endSeconds) + creationTimeMs
+ }
+ val rangeStart = valueAtSecond(startSeconds, rowsPerSecond, rampUpTimeSeconds)
+ val rangeEnd = valueAtSecond(endSeconds, rowsPerSecond, rampUpTimeSeconds)
+ logDebug(s"startSeconds: $startSeconds, endSeconds: $endSeconds, " +
+ s"rangeStart: $rangeStart, rangeEnd: $rangeEnd")
+
+ if (rangeStart == rangeEnd) {
+ return List.empty.asJava
+ }
+
+ val localStartTimeMs = creationTimeMs + TimeUnit.SECONDS.toMillis(startSeconds)
+ val relativeMsPerValue =
+ TimeUnit.SECONDS.toMillis(endSeconds - startSeconds).toDouble / (rangeEnd - rangeStart)
+ val numPartitions = {
+ val activeSession = SparkSession.getActiveSession
+ require(activeSession.isDefined)
+ Option(options.get(NUM_PARTITIONS).orElse(null.asInstanceOf[String]))
+ .map(_.toInt)
+ .getOrElse(activeSession.get.sparkContext.defaultParallelism)
+ }
+
+ (0 until numPartitions).map { p =>
+ new RateStreamMicroBatchInputPartition(
+ p, numPartitions, rangeStart, rangeEnd, localStartTimeMs, relativeMsPerValue)
+ : InputPartition[InternalRow]
+ }.toList.asJava
+ }
+
+ override def commit(end: Offset): Unit = {}
+
+ override def stop(): Unit = {}
+
+ override def toString: String = s"RateStreamV2[rowsPerSecond=$rowsPerSecond, " +
+ s"rampUpTimeSeconds=$rampUpTimeSeconds, " +
+ s"numPartitions=${options.get(NUM_PARTITIONS).orElse("default")}"
+}
+
+class RateStreamMicroBatchInputPartition(
+ partitionId: Int,
+ numPartitions: Int,
+ rangeStart: Long,
+ rangeEnd: Long,
+ localStartTimeMs: Long,
+ relativeMsPerValue: Double) extends InputPartition[InternalRow] {
+
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new RateStreamMicroBatchInputPartitionReader(partitionId, numPartitions, rangeStart, rangeEnd,
+ localStartTimeMs, relativeMsPerValue)
+}
+
+class RateStreamMicroBatchInputPartitionReader(
+ partitionId: Int,
+ numPartitions: Int,
+ rangeStart: Long,
+ rangeEnd: Long,
+ localStartTimeMs: Long,
+ relativeMsPerValue: Double) extends InputPartitionReader[InternalRow] {
+ private var count: Long = 0
+
+ override def next(): Boolean = {
+ rangeStart + partitionId + numPartitions * count < rangeEnd
+ }
+
+ override def get(): InternalRow = {
+ val currValue = rangeStart + partitionId + numPartitions * count
+ count += 1
+ val relative = math.round((currValue - rangeStart) * relativeMsPerValue)
+ InternalRow(DateTimeUtils.fromMillis(relative + localStartTimeMs), currValue)
+ }
+
+ override def close(): Unit = {}
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala
index 6942dfb..6bdd492 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala
@@ -17,11 +17,14 @@
package org.apache.spark.sql.execution.streaming.sources
+import java.util.Optional
+
import org.apache.spark.network.util.JavaUtils
-import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReadSupport
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReader
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.sources.v2._
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader}
import org.apache.spark.sql.types._
/**
@@ -39,12 +42,13 @@ import org.apache.spark.sql.types._
* be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.
*/
class RateStreamProvider extends DataSourceV2
- with MicroBatchReadSupportProvider with ContinuousReadSupportProvider with DataSourceRegister {
+ with MicroBatchReadSupport with ContinuousReadSupport with DataSourceRegister {
import RateStreamProvider._
- override def createMicroBatchReadSupport(
+ override def createMicroBatchReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): MicroBatchReadSupport = {
+ options: DataSourceOptions): MicroBatchReader = {
if (options.get(ROWS_PER_SECOND).isPresent) {
val rowsPerSecond = options.get(ROWS_PER_SECOND).get().toLong
if (rowsPerSecond <= 0) {
@@ -70,14 +74,17 @@ class RateStreamProvider extends DataSourceV2
}
}
- new RateStreamMicroBatchReadSupport(options, checkpointLocation)
+ if (schema.isPresent) {
+ throw new AnalysisException("The rate source does not support a user-specified schema.")
+ }
+
+ new RateStreamMicroBatchReader(options, checkpointLocation)
}
- override def createContinuousReadSupport(
+ override def createContinuousReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): ContinuousReadSupport = {
- new RateStreamContinuousReadSupport(options)
- }
+ options: DataSourceOptions): ContinuousReader = new RateStreamContinuousReader(options)
override def shortName(): String = "rate"
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala
index c50dc7b..cb76e86 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala
@@ -32,9 +32,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update}
import org.apache.spark.sql.execution.streaming.{MemorySinkBase, Sink}
-import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider}
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport}
import org.apache.spark.sql.sources.v2.writer._
-import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
@@ -42,15 +42,13 @@ 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 StreamingWriteSupportProvider
- with MemorySinkBase with Logging {
-
- override def createStreamingWriteSupport(
+class MemorySinkV2 extends DataSourceV2 with StreamWriteSupport with MemorySinkBase with Logging {
+ override def createStreamWriter(
queryId: String,
schema: StructType,
mode: OutputMode,
- options: DataSourceOptions): StreamingWriteSupport = {
- new MemoryStreamingWriteSupport(this, mode, schema)
+ options: DataSourceOptions): StreamWriter = {
+ new MemoryStreamWriter(this, mode, schema)
}
private case class AddedData(batchId: Long, data: Array[Row])
@@ -122,13 +120,10 @@ class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider
case class MemoryWriterCommitMessage(partition: Int, data: Seq[Row])
extends WriterCommitMessage {}
-class MemoryStreamingWriteSupport(
- val sink: MemorySinkV2, outputMode: OutputMode, schema: StructType)
- extends StreamingWriteSupport {
+class MemoryStreamWriter(val sink: MemorySinkV2, outputMode: OutputMode, schema: StructType)
+ extends StreamWriter {
- override def createStreamingWriterFactory: MemoryWriterFactory = {
- MemoryWriterFactory(outputMode, schema)
- }
+ override def createWriterFactory: MemoryWriterFactory = MemoryWriterFactory(outputMode, schema)
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
val newRows = messages.flatMap {
@@ -143,19 +138,13 @@ class MemoryStreamingWriteSupport(
}
case class MemoryWriterFactory(outputMode: OutputMode, schema: StructType)
- extends DataWriterFactory with StreamingDataWriterFactory {
+ extends DataWriterFactory[InternalRow] {
- override def createWriter(
- partitionId: Int,
- taskId: Long): DataWriter[InternalRow] = {
- new MemoryDataWriter(partitionId, outputMode, schema)
- }
-
- override def createWriter(
+ override def createDataWriter(
partitionId: Int,
taskId: Long,
epochId: Long): DataWriter[InternalRow] = {
- createWriter(partitionId, taskId)
+ new MemoryDataWriter(partitionId, outputMode, schema)
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
index b2a573e..874c479 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala
@@ -20,10 +20,11 @@ package org.apache.spark.sql.execution.streaming.sources
import java.io.{BufferedReader, InputStreamReader, IOException}
import java.net.Socket
import java.text.SimpleDateFormat
-import java.util.{Calendar, Locale}
+import java.util.{Calendar, List => JList, Locale, Optional}
import java.util.concurrent.atomic.AtomicBoolean
import javax.annotation.concurrent.GuardedBy
+import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import scala.util.{Failure, Success, Try}
@@ -31,15 +32,16 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.streaming.{LongOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder}
-import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousReadSupport
+import org.apache.spark.sql.execution.streaming.LongOffset
+import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousReader
import org.apache.spark.sql.sources.DataSourceRegister
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, DataSourceV2, MicroBatchReadSupportProvider}
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport, Offset}
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, DataSourceV2, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader, Offset}
import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType}
import org.apache.spark.unsafe.types.UTF8String
+// Shared object for micro-batch and continuous reader
object TextSocketReader {
val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil)
val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) ::
@@ -48,12 +50,14 @@ object TextSocketReader {
}
/**
- * A MicroBatchReadSupport that reads text lines through a TCP socket, designed only for tutorials
- * and debugging. This MicroBatchReadSupport will *not* work in production applications due to
- * multiple reasons, including no support for fault recovery.
+ * A MicroBatchReader that reads text lines through a TCP socket, designed only for tutorials and
+ * debugging. This MicroBatchReader will *not* work in production applications due to multiple
+ * reasons, including no support for fault recovery.
*/
-class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
- extends MicroBatchReadSupport with Logging {
+class TextSocketMicroBatchReader(options: DataSourceOptions) extends MicroBatchReader with Logging {
+
+ private var startOffset: Offset = _
+ private var endOffset: Offset = _
private val host: String = options.get("host").get()
private val port: Int = options.get("port").get().toInt
@@ -99,7 +103,7 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
logWarning(s"Stream closed by $host:$port")
return
}
- TextSocketMicroBatchReadSupport.this.synchronized {
+ TextSocketMicroBatchReader.this.synchronized {
val newData = (
UTF8String.fromString(line),
DateTimeUtils.fromMillis(Calendar.getInstance().getTimeInMillis)
@@ -116,15 +120,24 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
readThread.start()
}
- override def initialOffset(): Offset = LongOffset(-1L)
+ override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = synchronized {
+ startOffset = start.orElse(LongOffset(-1L))
+ endOffset = end.orElse(currentOffset)
+ }
- override def latestOffset(): Offset = currentOffset
+ override def getStartOffset(): Offset = {
+ Option(startOffset).getOrElse(throw new IllegalStateException("start offset not set"))
+ }
+
+ override def getEndOffset(): Offset = {
+ Option(endOffset).getOrElse(throw new IllegalStateException("end offset not set"))
+ }
override def deserializeOffset(json: String): Offset = {
LongOffset(json.toLong)
}
- override def fullSchema(): StructType = {
+ override def readSchema(): StructType = {
if (options.getBoolean("includeTimestamp", false)) {
TextSocketReader.SCHEMA_TIMESTAMP
} else {
@@ -132,14 +145,12 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
}
}
- override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
- }
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
+ assert(startOffset != null && endOffset != null,
+ "start offset and end offset should already be set before create read tasks.")
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val sc = config.asInstanceOf[SimpleStreamingScanConfig]
- val startOrdinal = sc.start.asInstanceOf[LongOffset].offset.toInt + 1
- val endOrdinal = sc.end.get.asInstanceOf[LongOffset].offset.toInt + 1
+ val startOrdinal = LongOffset.convert(startOffset).get.offset.toInt + 1
+ val endOrdinal = LongOffset.convert(endOffset).get.offset.toInt + 1
// Internal buffer only holds the batches after lastOffsetCommitted
val rawList = synchronized {
@@ -161,29 +172,26 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
slices(idx % numPartitions).append(r)
}
- slices.map(TextSocketInputPartition)
- }
+ (0 until numPartitions).map { i =>
+ val slice = slices(i)
+ new InputPartition[InternalRow] {
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new InputPartitionReader[InternalRow] {
+ private var currentIdx = -1
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- new PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val slice = partition.asInstanceOf[TextSocketInputPartition].slice
- new PartitionReader[InternalRow] {
- private var currentIdx = -1
+ override def next(): Boolean = {
+ currentIdx += 1
+ currentIdx < slice.size
+ }
- override def next(): Boolean = {
- currentIdx += 1
- currentIdx < slice.size
- }
+ override def get(): InternalRow = {
+ InternalRow(slice(currentIdx)._1, slice(currentIdx)._2)
+ }
- override def get(): InternalRow = {
- InternalRow(slice(currentIdx)._1, slice(currentIdx)._2)
+ override def close(): Unit = {}
}
-
- override def close(): Unit = {}
- }
}
- }
+ }.toList.asJava
}
override def commit(end: Offset): Unit = synchronized {
@@ -219,11 +227,8 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions)
override def toString: String = s"TextSocketV2[host: $host, port: $port]"
}
-case class TextSocketInputPartition(slice: ListBuffer[(UTF8String, Long)]) extends InputPartition
-
class TextSocketSourceProvider extends DataSourceV2
- with MicroBatchReadSupportProvider with ContinuousReadSupportProvider
- with DataSourceRegister with Logging {
+ with MicroBatchReadSupport with ContinuousReadSupport with DataSourceRegister with Logging {
private def checkParameters(params: DataSourceOptions): Unit = {
logWarning("The socket source should not be used for production applications! " +
@@ -243,18 +248,27 @@ class TextSocketSourceProvider extends DataSourceV2
}
}
- override def createMicroBatchReadSupport(
+ override def createMicroBatchReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): MicroBatchReadSupport = {
+ options: DataSourceOptions): MicroBatchReader = {
checkParameters(options)
- new TextSocketMicroBatchReadSupport(options)
+ if (schema.isPresent) {
+ throw new AnalysisException("The socket source does not support a user-specified schema.")
+ }
+
+ new TextSocketMicroBatchReader(options)
}
- override def createContinuousReadSupport(
+ override def createContinuousReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): ContinuousReadSupport = {
+ options: DataSourceOptions): ContinuousReader = {
checkParameters(options)
- new TextSocketContinuousReadSupport(options)
+ if (schema.isPresent) {
+ throw new AnalysisException("The socket source does not support a user-specified schema.")
+ }
+ new TextSocketContinuousReader(options)
}
/** String that represents the format that this data source provider uses. */
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
index 2a4db4a..7eb5db5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.streaming
-import java.util.Locale
+import java.util.{Locale, Optional}
import scala.collection.JavaConverters._
@@ -28,8 +28,8 @@ import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2}
import org.apache.spark.sql.sources.StreamSourceProvider
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider}
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
@@ -172,21 +172,19 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
case _ => None
}
ds match {
- case s: MicroBatchReadSupportProvider =>
- var tempReadSupport: MicroBatchReadSupport = null
+ case s: MicroBatchReadSupport =>
+ var tempReader: MicroBatchReader = null
val schema = try {
- val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath
- tempReadSupport = if (userSpecifiedSchema.isDefined) {
- s.createMicroBatchReadSupport(userSpecifiedSchema.get, tmpCheckpointPath, options)
- } else {
- s.createMicroBatchReadSupport(tmpCheckpointPath, options)
- }
- tempReadSupport.fullSchema()
+ tempReader = s.createMicroBatchReader(
+ Optional.ofNullable(userSpecifiedSchema.orNull),
+ Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath,
+ options)
+ tempReader.readSchema()
} finally {
// Stop tempReader to avoid side-effect thing
- if (tempReadSupport != null) {
- tempReadSupport.stop()
- tempReadSupport = null
+ if (tempReader != null) {
+ tempReader.stop()
+ tempReader = null
}
}
Dataset.ofRows(
@@ -194,28 +192,16 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
StreamingRelationV2(
s, source, extraOptions.toMap,
schema.toAttributes, v1Relation)(sparkSession))
- case s: ContinuousReadSupportProvider =>
- var tempReadSupport: ContinuousReadSupport = null
- val schema = try {
- val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath
- tempReadSupport = if (userSpecifiedSchema.isDefined) {
- s.createContinuousReadSupport(userSpecifiedSchema.get, tmpCheckpointPath, options)
- } else {
- s.createContinuousReadSupport(tmpCheckpointPath, options)
- }
- tempReadSupport.fullSchema()
- } finally {
- // Stop tempReader to avoid side-effect thing
- if (tempReadSupport != null) {
- tempReadSupport.stop()
- tempReadSupport = null
- }
- }
+ case s: ContinuousReadSupport =>
+ val tempReader = s.createContinuousReader(
+ Optional.ofNullable(userSpecifiedSchema.orNull),
+ Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath,
+ options)
Dataset.ofRows(
sparkSession,
StreamingRelationV2(
s, source, extraOptions.toMap,
- schema.toAttributes, v1Relation)(sparkSession))
+ tempReader.readSchema().toAttributes, v1Relation)(sparkSession))
case _ =>
// Code path for data source v1.
Dataset.ofRows(sparkSession, StreamingRelation(v1DataSource))
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
index 7866e4f..3b9a56f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
@@ -21,7 +21,7 @@ import java.util.Locale
import scala.collection.JavaConverters._
-import org.apache.spark.annotation.InterfaceStability
+import org.apache.spark.annotation.{InterfaceStability, Since}
import org.apache.spark.api.java.function.VoidFunction2
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes
@@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
import org.apache.spark.sql.execution.streaming.sources._
-import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider
+import org.apache.spark.sql.sources.v2.StreamWriteSupport
/**
* Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems,
@@ -270,7 +270,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
query
} else if (source == "foreach") {
assertNotPartitioned("foreach")
- val sink = ForeachWriteSupportProvider[T](foreachWriter, ds.exprEnc)
+ val sink = ForeachWriterProvider[T](foreachWriter, ds.exprEnc)
df.sparkSession.sessionState.streamingQueryManager.startQuery(
extraOptions.get("queryName"),
extraOptions.get("checkpointLocation"),
@@ -299,8 +299,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) {
val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf)
val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",")
val sink = ds.newInstance() match {
- case w: StreamingWriteSupportProvider
- if !disabledSources.contains(w.getClass.getCanonicalName) => w
+ case w: StreamWriteSupport if !disabledSources.contains(w.getClass.getCanonicalName) => w
case _ =>
val ds = DataSource(
df.sparkSession,
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
index cd52d99..25bb052 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
@@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution,
import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS
-import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider
+import org.apache.spark.sql.sources.v2.StreamWriteSupport
import org.apache.spark.util.{Clock, SystemClock, Utils}
/**
@@ -256,7 +256,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
}
(sink, trigger) match {
- case (v2Sink: StreamingWriteSupportProvider, trigger: ContinuousTrigger) =>
+ case (v2Sink: StreamWriteSupport, trigger: ContinuousTrigger) =>
if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) {
UnsupportedOperationChecker.checkForContinuous(analyzedPlan, outputMode)
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java
index 5602310..e4cead9 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java
@@ -24,71 +24,29 @@ import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.sources.Filter;
import org.apache.spark.sql.sources.GreaterThan;
-import org.apache.spark.sql.sources.v2.BatchReadSupportProvider;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.ReadSupport;
import org.apache.spark.sql.sources.v2.reader.*;
import org.apache.spark.sql.types.StructType;
-public class JavaAdvancedDataSourceV2 implements DataSourceV2, BatchReadSupportProvider {
+public class JavaAdvancedDataSourceV2 implements DataSourceV2, ReadSupport {
- public class ReadSupport extends JavaSimpleReadSupport {
- @Override
- public ScanConfigBuilder newScanConfigBuilder() {
- return new AdvancedScanConfigBuilder();
- }
-
- @Override
- public InputPartition[] planInputPartitions(ScanConfig config) {
- Filter[] filters = ((AdvancedScanConfigBuilder) config).filters;
- List<InputPartition> res = new ArrayList<>();
-
- Integer lowerBound = null;
- for (Filter filter : filters) {
- if (filter instanceof GreaterThan) {
- GreaterThan f = (GreaterThan) filter;
- if ("i".equals(f.attribute()) && f.value() instanceof Integer) {
- lowerBound = (Integer) f.value();
- break;
- }
- }
- }
-
- if (lowerBound == null) {
- res.add(new JavaRangeInputPartition(0, 5));
- res.add(new JavaRangeInputPartition(5, 10));
- } else if (lowerBound < 4) {
- res.add(new JavaRangeInputPartition(lowerBound + 1, 5));
- res.add(new JavaRangeInputPartition(5, 10));
- } else if (lowerBound < 9) {
- res.add(new JavaRangeInputPartition(lowerBound + 1, 10));
- }
-
- return res.stream().toArray(InputPartition[]::new);
- }
-
- @Override
- public PartitionReaderFactory createReaderFactory(ScanConfig config) {
- StructType requiredSchema = ((AdvancedScanConfigBuilder) config).requiredSchema;
- return new AdvancedReaderFactory(requiredSchema);
- }
- }
-
- public static class AdvancedScanConfigBuilder implements ScanConfigBuilder, ScanConfig,
- SupportsPushDownFilters, SupportsPushDownRequiredColumns {
+ public class Reader implements DataSourceReader, SupportsPushDownRequiredColumns,
+ SupportsPushDownFilters {
// Exposed for testing.
public StructType requiredSchema = new StructType().add("i", "int").add("j", "int");
public Filter[] filters = new Filter[0];
@Override
- public void pruneColumns(StructType requiredSchema) {
- this.requiredSchema = requiredSchema;
+ public StructType readSchema() {
+ return requiredSchema;
}
@Override
- public StructType readSchema() {
- return requiredSchema;
+ public void pruneColumns(StructType requiredSchema) {
+ this.requiredSchema = requiredSchema;
}
@Override
@@ -121,54 +79,79 @@ public class JavaAdvancedDataSourceV2 implements DataSourceV2, BatchReadSupportP
}
@Override
- public ScanConfig build() {
- return this;
+ public List<InputPartition<InternalRow>> planInputPartitions() {
+ List<InputPartition<InternalRow>> res = new ArrayList<>();
+
+ Integer lowerBound = null;
+ for (Filter filter : filters) {
+ if (filter instanceof GreaterThan) {
+ GreaterThan f = (GreaterThan) filter;
+ if ("i".equals(f.attribute()) && f.value() instanceof Integer) {
+ lowerBound = (Integer) f.value();
+ break;
+ }
+ }
+ }
+
+ if (lowerBound == null) {
+ res.add(new JavaAdvancedInputPartition(0, 5, requiredSchema));
+ res.add(new JavaAdvancedInputPartition(5, 10, requiredSchema));
+ } else if (lowerBound < 4) {
+ res.add(new JavaAdvancedInputPartition(lowerBound + 1, 5, requiredSchema));
+ res.add(new JavaAdvancedInputPartition(5, 10, requiredSchema));
+ } else if (lowerBound < 9) {
+ res.add(new JavaAdvancedInputPartition(lowerBound + 1, 10, requiredSchema));
+ }
+
+ return res;
}
}
- static class AdvancedReaderFactory implements PartitionReaderFactory {
- StructType requiredSchema;
+ static class JavaAdvancedInputPartition implements InputPartition<InternalRow>,
+ InputPartitionReader<InternalRow> {
+ private int start;
+ private int end;
+ private StructType requiredSchema;
- AdvancedReaderFactory(StructType requiredSchema) {
+ JavaAdvancedInputPartition(int start, int end, StructType requiredSchema) {
+ this.start = start;
+ this.end = end;
this.requiredSchema = requiredSchema;
}
@Override
- public PartitionReader<InternalRow> createReader(InputPartition partition) {
- JavaRangeInputPartition p = (JavaRangeInputPartition) partition;
- return new PartitionReader<InternalRow>() {
- private int current = p.start - 1;
-
- @Override
- public boolean next() throws IOException {
- current += 1;
- return current < p.end;
- }
+ public InputPartitionReader<InternalRow> createPartitionReader() {
+ return new JavaAdvancedInputPartition(start - 1, end, requiredSchema);
+ }
- @Override
- public InternalRow get() {
- Object[] values = new Object[requiredSchema.size()];
- for (int i = 0; i < values.length; i++) {
- if ("i".equals(requiredSchema.apply(i).name())) {
- values[i] = current;
- } else if ("j".equals(requiredSchema.apply(i).name())) {
- values[i] = -current;
- }
- }
- return new GenericInternalRow(values);
+ @Override
+ public boolean next() {
+ start += 1;
+ return start < end;
+ }
+
+ @Override
+ public InternalRow get() {
+ Object[] values = new Object[requiredSchema.size()];
+ for (int i = 0; i < values.length; i++) {
+ if ("i".equals(requiredSchema.apply(i).name())) {
+ values[i] = start;
+ } else if ("j".equals(requiredSchema.apply(i).name())) {
+ values[i] = -start;
}
+ }
+ return new GenericInternalRow(values);
+ }
- @Override
- public void close() throws IOException {
+ @Override
+ public void close() throws IOException {
- }
- };
}
}
@Override
- public BatchReadSupport createBatchReadSupport(DataSourceOptions options) {
- return new ReadSupport();
+ public DataSourceReader createReader(DataSourceOptions options) {
+ return new Reader();
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java
new file mode 100644
index 0000000..97d6176
--- /dev/null
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package test.org.apache.spark.sql.sources.v2;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.reader.*;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.vectorized.ColumnVector;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+
+public class JavaBatchDataSourceV2 implements DataSourceV2, ReadSupport {
+
+ class Reader implements DataSourceReader, SupportsScanColumnarBatch {
+ private final StructType schema = new StructType().add("i", "int").add("j", "int");
+
+ @Override
+ public StructType readSchema() {
+ return schema;
+ }
+
+ @Override
+ public List<InputPartition<ColumnarBatch>> planBatchInputPartitions() {
+ return java.util.Arrays.asList(
+ new JavaBatchInputPartition(0, 50), new JavaBatchInputPartition(50, 90));
+ }
+ }
+
+ static class JavaBatchInputPartition
+ implements InputPartition<ColumnarBatch>, InputPartitionReader<ColumnarBatch> {
+ private int start;
+ private int end;
+
+ private static final int BATCH_SIZE = 20;
+
+ private OnHeapColumnVector i;
+ private OnHeapColumnVector j;
+ private ColumnarBatch batch;
+
+ JavaBatchInputPartition(int start, int end) {
+ this.start = start;
+ this.end = end;
+ }
+
+ @Override
+ public InputPartitionReader<ColumnarBatch> createPartitionReader() {
+ this.i = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType);
+ this.j = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType);
+ ColumnVector[] vectors = new ColumnVector[2];
+ vectors[0] = i;
+ vectors[1] = j;
+ this.batch = new ColumnarBatch(vectors);
+ return this;
+ }
+
+ @Override
+ public boolean next() {
+ i.reset();
+ j.reset();
+ int count = 0;
+ while (start < end && count < BATCH_SIZE) {
+ i.putInt(count, start);
+ j.putInt(count, -start);
+ start += 1;
+ count += 1;
+ }
+
+ if (count == 0) {
+ return false;
+ } else {
+ batch.setNumRows(count);
+ return true;
+ }
+ }
+
+ @Override
+ public ColumnarBatch get() {
+ return batch;
+ }
+
+ @Override
+ public void close() throws IOException {
+ batch.close();
+ }
+ }
+
+
+ @Override
+ public DataSourceReader createReader(DataSourceOptions options) {
+ return new Reader();
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java
deleted file mode 100644
index 28a9330..0000000
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.sql.sources.v2;
-
-import java.io.IOException;
-
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector;
-import org.apache.spark.sql.sources.v2.BatchReadSupportProvider;
-import org.apache.spark.sql.sources.v2.DataSourceOptions;
-import org.apache.spark.sql.sources.v2.DataSourceV2;
-import org.apache.spark.sql.sources.v2.reader.*;
-import org.apache.spark.sql.types.DataTypes;
-import org.apache.spark.sql.vectorized.ColumnVector;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-
-public class JavaColumnarDataSourceV2 implements DataSourceV2, BatchReadSupportProvider {
-
- class ReadSupport extends JavaSimpleReadSupport {
-
- @Override
- public InputPartition[] planInputPartitions(ScanConfig config) {
- InputPartition[] partitions = new InputPartition[2];
- partitions[0] = new JavaRangeInputPartition(0, 50);
- partitions[1] = new JavaRangeInputPartition(50, 90);
- return partitions;
- }
-
- @Override
- public PartitionReaderFactory createReaderFactory(ScanConfig config) {
- return new ColumnarReaderFactory();
- }
- }
-
- static class ColumnarReaderFactory implements PartitionReaderFactory {
- private static final int BATCH_SIZE = 20;
-
- @Override
- public boolean supportColumnarReads(InputPartition partition) {
- return true;
- }
-
- @Override
- public PartitionReader<InternalRow> createReader(InputPartition partition) {
- throw new UnsupportedOperationException("");
- }
-
- @Override
- public PartitionReader<ColumnarBatch> createColumnarReader(InputPartition partition) {
- JavaRangeInputPartition p = (JavaRangeInputPartition) partition;
- OnHeapColumnVector i = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType);
- OnHeapColumnVector j = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType);
- ColumnVector[] vectors = new ColumnVector[2];
- vectors[0] = i;
- vectors[1] = j;
- ColumnarBatch batch = new ColumnarBatch(vectors);
-
- return new PartitionReader<ColumnarBatch>() {
- private int current = p.start;
-
- @Override
- public boolean next() throws IOException {
- i.reset();
- j.reset();
- int count = 0;
- while (current < p.end && count < BATCH_SIZE) {
- i.putInt(count, current);
- j.putInt(count, -current);
- current += 1;
- count += 1;
- }
-
- if (count == 0) {
- return false;
- } else {
- batch.setNumRows(count);
- return true;
- }
- }
-
- @Override
- public ColumnarBatch get() {
- return batch;
- }
-
- @Override
- public void close() throws IOException {
- batch.close();
- }
- };
- }
- }
-
- @Override
- public BatchReadSupport createBatchReadSupport(DataSourceOptions options) {
- return new ReadSupport();
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java
index 18a11dd..2d21324 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java
@@ -19,34 +19,38 @@ package test.org.apache.spark.sql.sources.v2;
import java.io.IOException;
import java.util.Arrays;
+import java.util.List;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.sources.v2.*;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.ReadSupport;
import org.apache.spark.sql.sources.v2.reader.*;
import org.apache.spark.sql.sources.v2.reader.partitioning.ClusteredDistribution;
import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution;
import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning;
+import org.apache.spark.sql.types.StructType;
-public class JavaPartitionAwareDataSource implements DataSourceV2, BatchReadSupportProvider {
+public class JavaPartitionAwareDataSource implements DataSourceV2, ReadSupport {
- class ReadSupport extends JavaSimpleReadSupport implements SupportsReportPartitioning {
+ class Reader implements DataSourceReader, SupportsReportPartitioning {
+ private final StructType schema = new StructType().add("a", "int").add("b", "int");
@Override
- public InputPartition[] planInputPartitions(ScanConfig config) {
- InputPartition[] partitions = new InputPartition[2];
- partitions[0] = new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6});
- partitions[1] = new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2});
- return partitions;
+ public StructType readSchema() {
+ return schema;
}
@Override
- public PartitionReaderFactory createReaderFactory(ScanConfig config) {
- return new SpecificReaderFactory();
+ public List<InputPartition<InternalRow>> planInputPartitions() {
+ return java.util.Arrays.asList(
+ new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6}),
+ new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2}));
}
@Override
- public Partitioning outputPartitioning(ScanConfig config) {
+ public Partitioning outputPartitioning() {
return new MyPartitioning();
}
}
@@ -62,53 +66,50 @@ public class JavaPartitionAwareDataSource implements DataSourceV2, BatchReadSupp
public boolean satisfy(Distribution distribution) {
if (distribution instanceof ClusteredDistribution) {
String[] clusteredCols = ((ClusteredDistribution) distribution).clusteredColumns;
- return Arrays.asList(clusteredCols).contains("i");
+ return Arrays.asList(clusteredCols).contains("a");
}
return false;
}
}
- static class SpecificInputPartition implements InputPartition {
- int[] i;
- int[] j;
+ static class SpecificInputPartition implements InputPartition<InternalRow>,
+ InputPartitionReader<InternalRow> {
+
+ private int[] i;
+ private int[] j;
+ private int current = -1;
SpecificInputPartition(int[] i, int[] j) {
assert i.length == j.length;
this.i = i;
this.j = j;
}
- }
- static class SpecificReaderFactory implements PartitionReaderFactory {
+ @Override
+ public boolean next() throws IOException {
+ current += 1;
+ return current < i.length;
+ }
+
+ @Override
+ public InternalRow get() {
+ return new GenericInternalRow(new Object[] {i[current], j[current]});
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
@Override
- public PartitionReader<InternalRow> createReader(InputPartition partition) {
- SpecificInputPartition p = (SpecificInputPartition) partition;
- return new PartitionReader<InternalRow>() {
- private int current = -1;
-
- @Override
- public boolean next() throws IOException {
- current += 1;
- return current < p.i.length;
- }
-
- @Override
- public InternalRow get() {
- return new GenericInternalRow(new Object[] {p.i[current], p.j[current]});
- }
-
- @Override
- public void close() throws IOException {
-
- }
- };
+ public InputPartitionReader<InternalRow> createPartitionReader() {
+ return this;
}
}
@Override
- public BatchReadSupport createBatchReadSupport(DataSourceOptions options) {
- return new ReadSupport();
+ public DataSourceReader createReader(DataSourceOptions options) {
+ return new Reader();
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java
index cc9ac04..6fd6a44 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java
@@ -17,39 +17,43 @@
package test.org.apache.spark.sql.sources.v2;
-import org.apache.spark.sql.sources.v2.BatchReadSupportProvider;
+import java.util.List;
+
+import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
import org.apache.spark.sql.sources.v2.DataSourceV2;
-import org.apache.spark.sql.sources.v2.reader.*;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
import org.apache.spark.sql.types.StructType;
-public class JavaSchemaRequiredDataSource implements DataSourceV2, BatchReadSupportProvider {
+public class JavaSchemaRequiredDataSource implements DataSourceV2, ReadSupport {
- class ReadSupport extends JavaSimpleReadSupport {
+ class Reader implements DataSourceReader {
private final StructType schema;
- ReadSupport(StructType schema) {
+ Reader(StructType schema) {
this.schema = schema;
}
@Override
- public StructType fullSchema() {
+ public StructType readSchema() {
return schema;
}
@Override
- public InputPartition[] planInputPartitions(ScanConfig config) {
- return new InputPartition[0];
+ public List<InputPartition<InternalRow>> planInputPartitions() {
+ return java.util.Collections.emptyList();
}
}
@Override
- public BatchReadSupport createBatchReadSupport(DataSourceOptions options) {
+ public DataSourceReader createReader(DataSourceOptions options) {
throw new IllegalArgumentException("requires a user-supplied schema");
}
@Override
- public BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) {
- return new ReadSupport(schema);
+ public DataSourceReader createReader(StructType schema, DataSourceOptions options) {
+ return new Reader(schema);
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java
index 2cdbba8..274dc37 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java
@@ -17,26 +17,72 @@
package test.org.apache.spark.sql.sources.v2;
-import org.apache.spark.sql.sources.v2.BatchReadSupportProvider;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.sources.v2.DataSourceV2;
import org.apache.spark.sql.sources.v2.DataSourceOptions;
-import org.apache.spark.sql.sources.v2.reader.*;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.types.StructType;
+
+public class JavaSimpleDataSourceV2 implements DataSourceV2, ReadSupport {
+
+ class Reader implements DataSourceReader {
+ private final StructType schema = new StructType().add("i", "int").add("j", "int");
+
+ @Override
+ public StructType readSchema() {
+ return schema;
+ }
+
+ @Override
+ public List<InputPartition<InternalRow>> planInputPartitions() {
+ return java.util.Arrays.asList(
+ new JavaSimpleInputPartition(0, 5),
+ new JavaSimpleInputPartition(5, 10));
+ }
+ }
+
+ static class JavaSimpleInputPartition implements InputPartition<InternalRow>,
+ InputPartitionReader<InternalRow> {
-public class JavaSimpleDataSourceV2 implements DataSourceV2, BatchReadSupportProvider {
+ private int start;
+ private int end;
- class ReadSupport extends JavaSimpleReadSupport {
+ JavaSimpleInputPartition(int start, int end) {
+ this.start = start;
+ this.end = end;
+ }
+
+ @Override
+ public InputPartitionReader<InternalRow> createPartitionReader() {
+ return new JavaSimpleInputPartition(start - 1, end);
+ }
@Override
- public InputPartition[] planInputPartitions(ScanConfig config) {
- InputPartition[] partitions = new InputPartition[2];
- partitions[0] = new JavaRangeInputPartition(0, 5);
- partitions[1] = new JavaRangeInputPartition(5, 10);
- return partitions;
+ public boolean next() {
+ start += 1;
+ return start < end;
+ }
+
+ @Override
+ public InternalRow get() {
+ return new GenericInternalRow(new Object[] {start, -start});
+ }
+
+ @Override
+ public void close() throws IOException {
+
}
}
@Override
- public BatchReadSupport createBatchReadSupport(DataSourceOptions options) {
- return new ReadSupport();
+ public DataSourceReader createReader(DataSourceOptions options) {
+ return new Reader();
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java
----------------------------------------------------------------------
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java
deleted file mode 100644
index 685f9b9..0000000
--- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package test.org.apache.spark.sql.sources.v2;
-
-import java.io.IOException;
-
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.sources.v2.reader.*;
-import org.apache.spark.sql.types.StructType;
-
-abstract class JavaSimpleReadSupport implements BatchReadSupport {
-
- @Override
- public StructType fullSchema() {
- return new StructType().add("i", "int").add("j", "int");
- }
-
- @Override
- public ScanConfigBuilder newScanConfigBuilder() {
- return new JavaNoopScanConfigBuilder(fullSchema());
- }
-
- @Override
- public PartitionReaderFactory createReaderFactory(ScanConfig config) {
- return new JavaSimpleReaderFactory();
- }
-}
-
-class JavaNoopScanConfigBuilder implements ScanConfigBuilder, ScanConfig {
-
- private StructType schema;
-
- JavaNoopScanConfigBuilder(StructType schema) {
- this.schema = schema;
- }
-
- @Override
- public ScanConfig build() {
- return this;
- }
-
- @Override
- public StructType readSchema() {
- return schema;
- }
-}
-
-class JavaSimpleReaderFactory implements PartitionReaderFactory {
-
- @Override
- public PartitionReader<InternalRow> createReader(InputPartition partition) {
- JavaRangeInputPartition p = (JavaRangeInputPartition) partition;
- return new PartitionReader<InternalRow>() {
- private int current = p.start - 1;
-
- @Override
- public boolean next() throws IOException {
- current += 1;
- return current < p.end;
- }
-
- @Override
- public InternalRow get() {
- return new GenericInternalRow(new Object[] {current, -current});
- }
-
- @Override
- public void close() throws IOException {
-
- }
- };
- }
-}
-
-class JavaRangeInputPartition implements InputPartition {
- int start;
- int end;
-
- JavaRangeInputPartition(int start, int end) {
- this.start = start;
- this.end = end;
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index a36b0cf..46b38be 100644
--- a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -9,6 +9,6 @@ org.apache.spark.sql.streaming.sources.FakeReadMicroBatchOnly
org.apache.spark.sql.streaming.sources.FakeReadContinuousOnly
org.apache.spark.sql.streaming.sources.FakeReadBothModes
org.apache.spark.sql.streaming.sources.FakeReadNeitherMode
-org.apache.spark.sql.streaming.sources.FakeWriteSupportProvider
+org.apache.spark.sql.streaming.sources.FakeWrite
org.apache.spark.sql.streaming.sources.FakeNoWrite
-org.apache.spark.sql.streaming.sources.FakeWriteSupportProviderV1Fallback
+org.apache.spark.sql.streaming.sources.FakeWriteV1Fallback
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala
----------------------------------------------------------------------
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 6185736..7bb2cf5 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
@@ -43,7 +43,7 @@ class MemorySinkV2Suite extends StreamTest with BeforeAndAfter {
test("streaming writer") {
val sink = new MemorySinkV2
- val writeSupport = new MemoryStreamingWriteSupport(
+ val writeSupport = new MemoryStreamWriter(
sink, OutputMode.Append(), new StructType().add("i", "int"))
writeSupport.commit(0,
Array(
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[7/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
[SPARK-24882][SQL] Revert [] improve data source v2 API from branch 2.4
## What changes were proposed in this pull request?
As discussed in the dev list, we don't want to include https://github.com/apache/spark/pull/22009 in Spark 2.4, as it needs data source v2 users to change the implementation intensitively, while they need to change again in next release.
## How was this patch tested?
existing tests
Author: Wenchen Fan <we...@databricks.com>
Closes #22388 from cloud-fan/revert.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/15d2e9d7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/15d2e9d7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/15d2e9d7
Branch: refs/heads/branch-2.4
Commit: 15d2e9d7d2f0d5ecefd69bdc3f8a149670b05e79
Parents: 4c1428f
Author: Wenchen Fan <we...@databricks.com>
Authored: Wed Sep 12 11:25:24 2018 -0700
Committer: gatorsmile <ga...@gmail.com>
Committed: Wed Sep 12 11:25:24 2018 -0700
----------------------------------------------------------------------
.../kafka010/KafkaContinuousReadSupport.scala | 255 -----------
.../sql/kafka010/KafkaContinuousReader.scala | 248 +++++++++++
.../kafka010/KafkaMicroBatchReadSupport.scala | 377 ----------------
.../sql/kafka010/KafkaMicroBatchReader.scala | 382 ++++++++++++++++
.../sql/kafka010/KafkaSourceProvider.scala | 37 +-
.../spark/sql/kafka010/KafkaStreamWriter.scala | 118 +++++
.../kafka010/KafkaStreamingWriteSupport.scala | 118 -----
.../kafka010/KafkaContinuousSourceSuite.scala | 12 +-
.../sql/kafka010/KafkaContinuousTest.scala | 8 +-
.../kafka010/KafkaMicroBatchSourceSuite.scala | 35 +-
.../sources/v2/BatchReadSupportProvider.java | 61 ---
.../sources/v2/BatchWriteSupportProvider.java | 59 ---
.../sql/sources/v2/ContinuousReadSupport.java | 46 ++
.../v2/ContinuousReadSupportProvider.java | 70 ---
.../spark/sql/sources/v2/DataSourceV2.java | 10 +-
.../sql/sources/v2/MicroBatchReadSupport.java | 52 +++
.../v2/MicroBatchReadSupportProvider.java | 70 ---
.../spark/sql/sources/v2/ReadSupport.java | 65 +++
.../sql/sources/v2/SessionConfigSupport.java | 7 +-
.../sql/sources/v2/StreamWriteSupport.java | 52 +++
.../v2/StreamingWriteSupportProvider.java | 54 ---
.../spark/sql/sources/v2/WriteSupport.java | 53 +++
.../sql/sources/v2/reader/BatchReadSupport.java | 51 ---
.../v2/reader/ContinuousInputPartition.java | 35 ++
.../sql/sources/v2/reader/DataSourceReader.java | 75 ++++
.../sql/sources/v2/reader/InputPartition.java | 26 +-
.../sources/v2/reader/InputPartitionReader.java | 53 +++
.../sql/sources/v2/reader/PartitionReader.java | 49 ---
.../v2/reader/PartitionReaderFactory.java | 66 ---
.../sql/sources/v2/reader/ReadSupport.java | 50 ---
.../spark/sql/sources/v2/reader/ScanConfig.java | 45 --
.../sources/v2/reader/ScanConfigBuilder.java | 30 --
.../spark/sql/sources/v2/reader/Statistics.java | 2 +-
.../v2/reader/SupportsPushDownFilters.java | 6 +-
.../reader/SupportsPushDownRequiredColumns.java | 8 +-
.../v2/reader/SupportsReportPartitioning.java | 12 +-
.../v2/reader/SupportsReportStatistics.java | 14 +-
.../v2/reader/SupportsScanColumnarBatch.java | 53 +++
.../partitioning/ClusteredDistribution.java | 4 +-
.../v2/reader/partitioning/Distribution.java | 6 +-
.../v2/reader/partitioning/Partitioning.java | 5 +-
.../ContinuousInputPartitionReader.java | 36 ++
.../streaming/ContinuousPartitionReader.java | 37 --
.../ContinuousPartitionReaderFactory.java | 40 --
.../reader/streaming/ContinuousReadSupport.java | 77 ----
.../v2/reader/streaming/ContinuousReader.java | 79 ++++
.../reader/streaming/MicroBatchReadSupport.java | 60 ---
.../v2/reader/streaming/MicroBatchReader.java | 75 ++++
.../sql/sources/v2/reader/streaming/Offset.java | 4 +-
.../reader/streaming/StreamingReadSupport.java | 49 ---
.../sources/v2/writer/BatchWriteSupport.java | 101 -----
.../sql/sources/v2/writer/DataSourceWriter.java | 116 +++++
.../spark/sql/sources/v2/writer/DataWriter.java | 16 +-
.../sources/v2/writer/DataWriterFactory.java | 23 +-
.../sources/v2/writer/WriterCommitMessage.java | 9 +-
.../v2/writer/streaming/StreamWriter.java | 71 +++
.../streaming/StreamingDataWriterFactory.java | 59 ---
.../writer/streaming/StreamingWriteSupport.java | 71 ---
.../org/apache/spark/sql/DataFrameReader.scala | 4 +-
.../org/apache/spark/sql/DataFrameWriter.scala | 8 +-
.../datasources/v2/DataSourceRDD.scala | 44 +-
.../datasources/v2/DataSourceV2Relation.scala | 72 ++-
.../datasources/v2/DataSourceV2ScanExec.scala | 65 +--
.../datasources/v2/DataSourceV2Strategy.scala | 49 +--
.../datasources/v2/DataSourceV2Utils.scala | 9 -
.../v2/WriteToDataSourceV2Exec.scala | 40 +-
.../streaming/MicroBatchExecution.scala | 91 ++--
.../execution/streaming/ProgressReporter.scala | 8 +-
.../SimpleStreamingScanConfigBuilder.scala | 40 --
.../execution/streaming/StreamingRelation.scala | 6 +-
.../spark/sql/execution/streaming/console.scala | 14 +-
.../continuous/ContinuousDataSourceRDD.scala | 37 +-
.../continuous/ContinuousExecution.scala | 51 +--
.../continuous/ContinuousQueuedDataReader.scala | 29 +-
.../continuous/ContinuousRateStreamSource.scala | 60 +--
.../continuous/ContinuousTextSocketSource.scala | 72 ++-
.../continuous/ContinuousWriteRDD.scala | 7 +-
.../streaming/continuous/EpochCoordinator.scala | 18 +-
.../WriteToContinuousDataSource.scala | 4 +-
.../WriteToContinuousDataSourceExec.scala | 10 +-
.../spark/sql/execution/streaming/memory.scala | 51 ++-
.../streaming/sources/ConsoleWriteSupport.scala | 71 ---
.../streaming/sources/ConsoleWriter.scala | 72 +++
.../sources/ContinuousMemoryStream.scala | 76 ++--
.../sources/ForeachWriteSupportProvider.scala | 140 ------
.../sources/ForeachWriterProvider.scala | 139 ++++++
.../sources/MicroBatchWritSupport.scala | 51 ---
.../streaming/sources/MicroBatchWriter.scala | 37 ++
.../sources/PackedRowWriterFactory.scala | 9 +-
.../RateControlMicroBatchReadSupport.scala | 31 --
.../RateStreamMicroBatchReadSupport.scala | 215 ---------
.../sources/RateStreamMicroBatchReader.scala | 220 ++++++++++
.../streaming/sources/RateStreamProvider.scala | 27 +-
.../execution/streaming/sources/memoryV2.scala | 35 +-
.../execution/streaming/sources/socket.scala | 114 ++---
.../spark/sql/streaming/DataStreamReader.scala | 52 +--
.../spark/sql/streaming/DataStreamWriter.scala | 9 +-
.../sql/streaming/StreamingQueryManager.scala | 4 +-
.../sources/v2/JavaAdvancedDataSourceV2.java | 147 +++----
.../sql/sources/v2/JavaBatchDataSourceV2.java | 114 +++++
.../sources/v2/JavaColumnarDataSourceV2.java | 114 -----
.../v2/JavaPartitionAwareDataSource.java | 81 ++--
.../v2/JavaSchemaRequiredDataSource.java | 26 +-
.../sql/sources/v2/JavaSimpleDataSourceV2.java | 68 ++-
.../sql/sources/v2/JavaSimpleReadSupport.java | 99 -----
....apache.spark.sql.sources.DataSourceRegister | 4 +-
.../execution/streaming/MemorySinkV2Suite.scala | 2 +-
.../sources/ConsoleWriteSupportSuite.scala | 151 -------
.../streaming/sources/ConsoleWriterSuite.scala | 153 +++++++
.../sources/RateStreamProviderSuite.scala | 84 ++--
.../sources/TextSocketStreamSuite.scala | 81 ++--
.../sql/sources/v2/DataSourceV2Suite.scala | 438 +++++++++----------
.../sources/v2/SimpleWritableDataSource.scala | 110 +++--
.../apache/spark/sql/streaming/StreamTest.scala | 2 +-
.../streaming/StreamingQueryListenerSuite.scala | 4 +-
.../sql/streaming/StreamingQuerySuite.scala | 58 ++-
.../ContinuousQueuedDataReaderSuite.scala | 45 +-
.../streaming/continuous/ContinuousSuite.scala | 2 +-
.../continuous/EpochCoordinatorSuite.scala | 18 +-
.../sources/StreamingDataSourceV2Suite.scala | 95 ++--
120 files changed, 3619 insertions(+), 4070 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
deleted file mode 100644
index 1753a28..0000000
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.kafka010
-
-import java.{util => ju}
-import java.util.concurrent.TimeoutException
-
-import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException}
-import org.apache.kafka.common.TopicPartition
-
-import org.apache.spark.TaskContext
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming._
-import org.apache.spark.sql.types.StructType
-
-/**
- * A [[ContinuousReadSupport]] for data from kafka.
- *
- * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be
- * read by per-task consumers generated later.
- * @param kafkaParams String params for per-task Kafka consumers.
- * @param sourceOptions The [[org.apache.spark.sql.sources.v2.DataSourceOptions]] params which
- * are not Kafka consumer params.
- * @param metadataPath Path to a directory this reader can use for writing metadata.
- * @param initialOffsets The Kafka offsets to start reading data at.
- * @param failOnDataLoss Flag indicating whether reading should fail in data loss
- * scenarios, where some offsets after the specified initial ones can't be
- * properly read.
- */
-class KafkaContinuousReadSupport(
- offsetReader: KafkaOffsetReader,
- kafkaParams: ju.Map[String, Object],
- sourceOptions: Map[String, String],
- metadataPath: String,
- initialOffsets: KafkaOffsetRangeLimit,
- failOnDataLoss: Boolean)
- extends ContinuousReadSupport with Logging {
-
- private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong
-
- override def initialOffset(): Offset = {
- val offsets = initialOffsets match {
- case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
- case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets())
- case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
- }
- logInfo(s"Initial offsets: $offsets")
- offsets
- }
-
- override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema
-
- override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
- new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss)
- }
-
- override def deserializeOffset(json: String): Offset = {
- KafkaSourceOffset(JsonUtils.partitionOffsets(json))
- }
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets
- startOffsets.toSeq.map {
- case (topicPartition, start) =>
- KafkaContinuousInputPartition(
- topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss)
- }.toArray
- }
-
- override def createContinuousReaderFactory(
- config: ScanConfig): ContinuousPartitionReaderFactory = {
- KafkaContinuousReaderFactory
- }
-
- /** Stop this source and free any resources it has allocated. */
- def stop(): Unit = synchronized {
- offsetReader.close()
- }
-
- override def commit(end: Offset): Unit = {}
-
- override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = {
- val mergedMap = offsets.map {
- case KafkaSourcePartitionOffset(p, o) => Map(p -> o)
- }.reduce(_ ++ _)
- KafkaSourceOffset(mergedMap)
- }
-
- override def needsReconfiguration(config: ScanConfig): Boolean = {
- val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions
- offsetReader.fetchLatestOffsets().keySet != knownPartitions
- }
-
- override def toString(): String = s"KafkaSource[$offsetReader]"
-
- /**
- * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
- * Otherwise, just log a warning.
- */
- private def reportDataLoss(message: String): Unit = {
- if (failOnDataLoss) {
- throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE")
- } else {
- logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE")
- }
- }
-}
-
-/**
- * An input partition for continuous Kafka processing. This will be serialized and transformed
- * into a full reader on executors.
- *
- * @param topicPartition The (topic, partition) pair this task is responsible for.
- * @param startOffset The offset to start reading from within the partition.
- * @param kafkaParams Kafka consumer params to use.
- * @param pollTimeoutMs The timeout for Kafka consumer polling.
- * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets
- * are skipped.
- */
-case class KafkaContinuousInputPartition(
- topicPartition: TopicPartition,
- startOffset: Long,
- kafkaParams: ju.Map[String, Object],
- pollTimeoutMs: Long,
- failOnDataLoss: Boolean) extends InputPartition
-
-object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory {
- override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[KafkaContinuousInputPartition]
- new KafkaContinuousPartitionReader(
- p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss)
- }
-}
-
-class KafkaContinuousScanConfigBuilder(
- schema: StructType,
- startOffset: Offset,
- offsetReader: KafkaOffsetReader,
- reportDataLoss: String => Unit)
- extends ScanConfigBuilder {
-
- override def build(): ScanConfig = {
- val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset)
-
- val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
- val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
- val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)
-
- val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
- if (deletedPartitions.nonEmpty) {
- reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
- }
-
- val startOffsets = newPartitionOffsets ++
- oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_))
- KafkaContinuousScanConfig(schema, startOffsets)
- }
-}
-
-case class KafkaContinuousScanConfig(
- readSchema: StructType,
- startOffsets: Map[TopicPartition, Long])
- extends ScanConfig {
-
- // Created when building the scan config builder. If this diverges from the partitions at the
- // latest offsets, we need to reconfigure the kafka read support.
- def knownPartitions: Set[TopicPartition] = startOffsets.keySet
-}
-
-/**
- * A per-task data reader for continuous Kafka processing.
- *
- * @param topicPartition The (topic, partition) pair this data reader is responsible for.
- * @param startOffset The offset to start reading from within the partition.
- * @param kafkaParams Kafka consumer params to use.
- * @param pollTimeoutMs The timeout for Kafka consumer polling.
- * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets
- * are skipped.
- */
-class KafkaContinuousPartitionReader(
- topicPartition: TopicPartition,
- startOffset: Long,
- kafkaParams: ju.Map[String, Object],
- pollTimeoutMs: Long,
- failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] {
- private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache = false)
- private val converter = new KafkaRecordToUnsafeRowConverter
-
- private var nextKafkaOffset = startOffset
- private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _
-
- override def next(): Boolean = {
- var r: ConsumerRecord[Array[Byte], Array[Byte]] = null
- while (r == null) {
- if (TaskContext.get().isInterrupted() || TaskContext.get().isCompleted()) return false
- // Our consumer.get is not interruptible, so we have to set a low poll timeout, leaving
- // interrupt points to end the query rather than waiting for new data that might never come.
- try {
- r = consumer.get(
- nextKafkaOffset,
- untilOffset = Long.MaxValue,
- pollTimeoutMs,
- failOnDataLoss)
- } catch {
- // We didn't read within the timeout. We're supposed to block indefinitely for new data, so
- // swallow and ignore this.
- case _: TimeoutException | _: org.apache.kafka.common.errors.TimeoutException =>
-
- // This is a failOnDataLoss exception. Retry if nextKafkaOffset is within the data range,
- // or if it's the endpoint of the data range (i.e. the "true" next offset).
- case e: IllegalStateException if e.getCause.isInstanceOf[OffsetOutOfRangeException] =>
- val range = consumer.getAvailableOffsetRange()
- if (range.latest >= nextKafkaOffset && range.earliest <= nextKafkaOffset) {
- // retry
- } else {
- throw e
- }
- }
- }
- nextKafkaOffset = r.offset + 1
- currentRecord = r
- true
- }
-
- override def get(): UnsafeRow = {
- converter.toUnsafeRow(currentRecord)
- }
-
- override def getOffset(): KafkaSourcePartitionOffset = {
- KafkaSourcePartitionOffset(topicPartition, nextKafkaOffset)
- }
-
- override def close(): Unit = {
- consumer.release()
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
new file mode 100644
index 0000000..8ce56a2
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.kafka010
+
+import java.{util => ju}
+import java.util.concurrent.TimeoutException
+
+import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException}
+import org.apache.kafka.common.TopicPartition
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.reader._
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * A [[ContinuousReader]] for data from kafka.
+ *
+ * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be
+ * read by per-task consumers generated later.
+ * @param kafkaParams String params for per-task Kafka consumers.
+ * @param sourceOptions The [[org.apache.spark.sql.sources.v2.DataSourceOptions]] params which
+ * are not Kafka consumer params.
+ * @param metadataPath Path to a directory this reader can use for writing metadata.
+ * @param initialOffsets The Kafka offsets to start reading data at.
+ * @param failOnDataLoss Flag indicating whether reading should fail in data loss
+ * scenarios, where some offsets after the specified initial ones can't be
+ * properly read.
+ */
+class KafkaContinuousReader(
+ offsetReader: KafkaOffsetReader,
+ kafkaParams: ju.Map[String, Object],
+ sourceOptions: Map[String, String],
+ metadataPath: String,
+ initialOffsets: KafkaOffsetRangeLimit,
+ failOnDataLoss: Boolean)
+ extends ContinuousReader with Logging {
+
+ private lazy val session = SparkSession.getActiveSession.get
+ private lazy val sc = session.sparkContext
+
+ private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong
+
+ // Initialized when creating reader factories. If this diverges from the partitions at the latest
+ // offsets, we need to reconfigure.
+ // Exposed outside this object only for unit tests.
+ @volatile private[sql] var knownPartitions: Set[TopicPartition] = _
+
+ override def readSchema: StructType = KafkaOffsetReader.kafkaSchema
+
+ private var offset: Offset = _
+ override def setStartOffset(start: ju.Optional[Offset]): Unit = {
+ offset = start.orElse {
+ val offsets = initialOffsets match {
+ case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
+ case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets())
+ case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
+ }
+ logInfo(s"Initial offsets: $offsets")
+ offsets
+ }
+ }
+
+ override def getStartOffset(): Offset = offset
+
+ override def deserializeOffset(json: String): Offset = {
+ KafkaSourceOffset(JsonUtils.partitionOffsets(json))
+ }
+
+ override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
+ import scala.collection.JavaConverters._
+
+ val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset)
+
+ val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
+ val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
+ val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)
+
+ val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
+ if (deletedPartitions.nonEmpty) {
+ reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
+ }
+
+ val startOffsets = newPartitionOffsets ++
+ oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_))
+ knownPartitions = startOffsets.keySet
+
+ startOffsets.toSeq.map {
+ case (topicPartition, start) =>
+ KafkaContinuousInputPartition(
+ topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss
+ ): InputPartition[InternalRow]
+ }.asJava
+ }
+
+ /** Stop this source and free any resources it has allocated. */
+ def stop(): Unit = synchronized {
+ offsetReader.close()
+ }
+
+ override def commit(end: Offset): Unit = {}
+
+ override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = {
+ val mergedMap = offsets.map {
+ case KafkaSourcePartitionOffset(p, o) => Map(p -> o)
+ }.reduce(_ ++ _)
+ KafkaSourceOffset(mergedMap)
+ }
+
+ override def needsReconfiguration(): Boolean = {
+ knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions
+ }
+
+ override def toString(): String = s"KafkaSource[$offsetReader]"
+
+ /**
+ * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
+ * Otherwise, just log a warning.
+ */
+ private def reportDataLoss(message: String): Unit = {
+ if (failOnDataLoss) {
+ throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE")
+ } else {
+ logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE")
+ }
+ }
+}
+
+/**
+ * An input partition for continuous Kafka processing. This will be serialized and transformed
+ * into a full reader on executors.
+ *
+ * @param topicPartition The (topic, partition) pair this task is responsible for.
+ * @param startOffset The offset to start reading from within the partition.
+ * @param kafkaParams Kafka consumer params to use.
+ * @param pollTimeoutMs The timeout for Kafka consumer polling.
+ * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets
+ * are skipped.
+ */
+case class KafkaContinuousInputPartition(
+ topicPartition: TopicPartition,
+ startOffset: Long,
+ kafkaParams: ju.Map[String, Object],
+ pollTimeoutMs: Long,
+ failOnDataLoss: Boolean) extends ContinuousInputPartition[InternalRow] {
+
+ override def createContinuousReader(
+ offset: PartitionOffset): InputPartitionReader[InternalRow] = {
+ val kafkaOffset = offset.asInstanceOf[KafkaSourcePartitionOffset]
+ require(kafkaOffset.topicPartition == topicPartition,
+ s"Expected topicPartition: $topicPartition, but got: ${kafkaOffset.topicPartition}")
+ new KafkaContinuousInputPartitionReader(
+ topicPartition, kafkaOffset.partitionOffset, kafkaParams, pollTimeoutMs, failOnDataLoss)
+ }
+
+ override def createPartitionReader(): KafkaContinuousInputPartitionReader = {
+ new KafkaContinuousInputPartitionReader(
+ topicPartition, startOffset, kafkaParams, pollTimeoutMs, failOnDataLoss)
+ }
+}
+
+/**
+ * A per-task data reader for continuous Kafka processing.
+ *
+ * @param topicPartition The (topic, partition) pair this data reader is responsible for.
+ * @param startOffset The offset to start reading from within the partition.
+ * @param kafkaParams Kafka consumer params to use.
+ * @param pollTimeoutMs The timeout for Kafka consumer polling.
+ * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets
+ * are skipped.
+ */
+class KafkaContinuousInputPartitionReader(
+ topicPartition: TopicPartition,
+ startOffset: Long,
+ kafkaParams: ju.Map[String, Object],
+ pollTimeoutMs: Long,
+ failOnDataLoss: Boolean) extends ContinuousInputPartitionReader[InternalRow] {
+ private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache = false)
+ private val converter = new KafkaRecordToUnsafeRowConverter
+
+ private var nextKafkaOffset = startOffset
+ private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _
+
+ override def next(): Boolean = {
+ var r: ConsumerRecord[Array[Byte], Array[Byte]] = null
+ while (r == null) {
+ if (TaskContext.get().isInterrupted() || TaskContext.get().isCompleted()) return false
+ // Our consumer.get is not interruptible, so we have to set a low poll timeout, leaving
+ // interrupt points to end the query rather than waiting for new data that might never come.
+ try {
+ r = consumer.get(
+ nextKafkaOffset,
+ untilOffset = Long.MaxValue,
+ pollTimeoutMs,
+ failOnDataLoss)
+ } catch {
+ // We didn't read within the timeout. We're supposed to block indefinitely for new data, so
+ // swallow and ignore this.
+ case _: TimeoutException | _: org.apache.kafka.common.errors.TimeoutException =>
+
+ // This is a failOnDataLoss exception. Retry if nextKafkaOffset is within the data range,
+ // or if it's the endpoint of the data range (i.e. the "true" next offset).
+ case e: IllegalStateException if e.getCause.isInstanceOf[OffsetOutOfRangeException] =>
+ val range = consumer.getAvailableOffsetRange()
+ if (range.latest >= nextKafkaOffset && range.earliest <= nextKafkaOffset) {
+ // retry
+ } else {
+ throw e
+ }
+ }
+ }
+ nextKafkaOffset = r.offset + 1
+ currentRecord = r
+ true
+ }
+
+ override def get(): UnsafeRow = {
+ converter.toUnsafeRow(currentRecord)
+ }
+
+ override def getOffset(): KafkaSourcePartitionOffset = {
+ KafkaSourcePartitionOffset(topicPartition, nextKafkaOffset)
+ }
+
+ override def close(): Unit = {
+ consumer.release()
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
deleted file mode 100644
index bb4de67..0000000
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala
+++ /dev/null
@@ -1,377 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.kafka010
-
-import java.{util => ju}
-import java.io._
-import java.nio.charset.StandardCharsets
-
-import org.apache.commons.io.IOUtils
-
-import org.apache.spark.SparkEnv
-import org.apache.spark.internal.Logging
-import org.apache.spark.scheduler.ExecutorCacheTaskLocation
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder}
-import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchReadSupport
-import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
-import org.apache.spark.sql.sources.v2.DataSourceOptions
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset}
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.util.UninterruptibleThread
-
-/**
- * A [[MicroBatchReadSupport]] that reads data from Kafka.
- *
- * The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains
- * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For
- * example if the last record in a Kafka topic "t", partition 2 is offset 5, then
- * KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent
- * with the semantics of `KafkaConsumer.position()`.
- *
- * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user
- * must make sure all messages in a topic have been processed when deleting a topic.
- *
- * There is a known issue caused by KAFKA-1894: the query using Kafka maybe cannot be stopped.
- * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers
- * and not use wrong broker addresses.
- */
-private[kafka010] class KafkaMicroBatchReadSupport(
- kafkaOffsetReader: KafkaOffsetReader,
- executorKafkaParams: ju.Map[String, Object],
- options: DataSourceOptions,
- metadataPath: String,
- startingOffsets: KafkaOffsetRangeLimit,
- failOnDataLoss: Boolean) extends RateControlMicroBatchReadSupport with Logging {
-
- private val pollTimeoutMs = options.getLong(
- "kafkaConsumer.pollTimeoutMs",
- SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L)
-
- private val maxOffsetsPerTrigger =
- Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)
-
- private val rangeCalculator = KafkaOffsetRangeCalculator(options)
-
- private var endPartitionOffsets: KafkaSourceOffset = _
-
- /**
- * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only
- * called in StreamExecutionThread. Otherwise, interrupting a thread while running
- * `KafkaConsumer.poll` may hang forever (KAFKA-1894).
- */
- override def initialOffset(): Offset = {
- KafkaSourceOffset(getOrCreateInitialPartitionOffsets())
- }
-
- override def latestOffset(start: Offset): Offset = {
- val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
- val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
- endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets =>
- rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets)
- }.getOrElse {
- latestPartitionOffsets
- })
- endPartitionOffsets
- }
-
- override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema
-
- override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
- }
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val sc = config.asInstanceOf[SimpleStreamingScanConfig]
- val startPartitionOffsets = sc.start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
- val endPartitionOffsets = sc.end.get.asInstanceOf[KafkaSourceOffset].partitionToOffsets
-
- // Find the new partitions, and get their earliest offsets
- val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet)
- val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq)
- if (newPartitionInitialOffsets.keySet != newPartitions) {
- // We cannot get from offsets for some partitions. It means they got deleted.
- val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet)
- reportDataLoss(
- s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed")
- }
- logInfo(s"Partitions added: $newPartitionInitialOffsets")
- newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) =>
- reportDataLoss(
- s"Added partition $p starts from $o instead of 0. Some data may have been missed")
- }
-
- // Find deleted partitions, and report data loss if required
- val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
- if (deletedPartitions.nonEmpty) {
- reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
- }
-
- // Use the end partitions to calculate offset ranges to ignore partitions that have
- // been deleted
- val topicPartitions = endPartitionOffsets.keySet.filter { tp =>
- // Ignore partitions that we don't know the from offsets.
- newPartitionInitialOffsets.contains(tp) || startPartitionOffsets.contains(tp)
- }.toSeq
- logDebug("TopicPartitions: " + topicPartitions.mkString(", "))
-
- // Calculate offset ranges
- val offsetRanges = rangeCalculator.getRanges(
- fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets,
- untilOffsets = endPartitionOffsets,
- executorLocations = getSortedExecutorList())
-
- // Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions,
- // that is, concurrent tasks will not read the same TopicPartitions.
- val reuseKafkaConsumer = offsetRanges.map(_.topicPartition).toSet.size == offsetRanges.size
-
- // Generate factories based on the offset ranges
- offsetRanges.map { range =>
- KafkaMicroBatchInputPartition(
- range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer)
- }.toArray
- }
-
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- KafkaMicroBatchReaderFactory
- }
-
- override def deserializeOffset(json: String): Offset = {
- KafkaSourceOffset(JsonUtils.partitionOffsets(json))
- }
-
- override def commit(end: Offset): Unit = {}
-
- override def stop(): Unit = {
- kafkaOffsetReader.close()
- }
-
- override def toString(): String = s"KafkaV2[$kafkaOffsetReader]"
-
- /**
- * Read initial partition offsets from the checkpoint, or decide the offsets and write them to
- * the checkpoint.
- */
- private def getOrCreateInitialPartitionOffsets(): PartitionOffsetMap = {
- // Make sure that `KafkaConsumer.poll` is only called in StreamExecutionThread.
- // Otherwise, interrupting a thread while running `KafkaConsumer.poll` may hang forever
- // (KAFKA-1894).
- assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-
- // SparkSession is required for getting Hadoop configuration for writing to checkpoints
- assert(SparkSession.getActiveSession.nonEmpty)
-
- val metadataLog =
- new KafkaSourceInitialOffsetWriter(SparkSession.getActiveSession.get, metadataPath)
- metadataLog.get(0).getOrElse {
- val offsets = startingOffsets match {
- case EarliestOffsetRangeLimit =>
- KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets())
- case LatestOffsetRangeLimit =>
- KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets())
- case SpecificOffsetRangeLimit(p) =>
- kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss)
- }
- metadataLog.add(0, offsets)
- logInfo(s"Initial offsets: $offsets")
- offsets
- }.partitionToOffsets
- }
-
- /** Proportionally distribute limit number of offsets among topicpartitions */
- private def rateLimit(
- limit: Long,
- from: PartitionOffsetMap,
- until: PartitionOffsetMap): PartitionOffsetMap = {
- val fromNew = kafkaOffsetReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq)
- val sizes = until.flatMap {
- case (tp, end) =>
- // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it
- from.get(tp).orElse(fromNew.get(tp)).flatMap { begin =>
- val size = end - begin
- logDebug(s"rateLimit $tp size is $size")
- if (size > 0) Some(tp -> size) else None
- }
- }
- val total = sizes.values.sum.toDouble
- if (total < 1) {
- until
- } else {
- until.map {
- case (tp, end) =>
- tp -> sizes.get(tp).map { size =>
- val begin = from.get(tp).getOrElse(fromNew(tp))
- val prorate = limit * (size / total)
- // Don't completely starve small topicpartitions
- val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong
- // Paranoia, make sure not to return an offset that's past end
- Math.min(end, off)
- }.getOrElse(end)
- }
- }
- }
-
- private def getSortedExecutorList(): Array[String] = {
-
- def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = {
- if (a.host == b.host) {
- a.executorId > b.executorId
- } else {
- a.host > b.host
- }
- }
-
- val bm = SparkEnv.get.blockManager
- bm.master.getPeers(bm.blockManagerId).toArray
- .map(x => ExecutorCacheTaskLocation(x.host, x.executorId))
- .sortWith(compare)
- .map(_.toString)
- }
-
- /**
- * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
- * Otherwise, just log a warning.
- */
- private def reportDataLoss(message: String): Unit = {
- if (failOnDataLoss) {
- throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE")
- } else {
- logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE")
- }
- }
-
- /** A version of [[HDFSMetadataLog]] specialized for saving the initial offsets. */
- class KafkaSourceInitialOffsetWriter(sparkSession: SparkSession, metadataPath: String)
- extends HDFSMetadataLog[KafkaSourceOffset](sparkSession, metadataPath) {
-
- val VERSION = 1
-
- override def serialize(metadata: KafkaSourceOffset, out: OutputStream): Unit = {
- out.write(0) // A zero byte is written to support Spark 2.1.0 (SPARK-19517)
- val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
- writer.write("v" + VERSION + "\n")
- writer.write(metadata.json)
- writer.flush
- }
-
- override def deserialize(in: InputStream): KafkaSourceOffset = {
- in.read() // A zero byte is read to support Spark 2.1.0 (SPARK-19517)
- val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
- // HDFSMetadataLog guarantees that it never creates a partial file.
- assert(content.length != 0)
- if (content(0) == 'v') {
- val indexOfNewLine = content.indexOf("\n")
- if (indexOfNewLine > 0) {
- val version = parseVersion(content.substring(0, indexOfNewLine), VERSION)
- KafkaSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1)))
- } else {
- throw new IllegalStateException(
- s"Log file was malformed: failed to detect the log file version line.")
- }
- } else {
- // The log was generated by Spark 2.1.0
- KafkaSourceOffset(SerializedOffset(content))
- }
- }
- }
-}
-
-/** A [[InputPartition]] for reading Kafka data in a micro-batch streaming query. */
-private[kafka010] case class KafkaMicroBatchInputPartition(
- offsetRange: KafkaOffsetRange,
- executorKafkaParams: ju.Map[String, Object],
- pollTimeoutMs: Long,
- failOnDataLoss: Boolean,
- reuseKafkaConsumer: Boolean) extends InputPartition
-
-private[kafka010] object KafkaMicroBatchReaderFactory extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[KafkaMicroBatchInputPartition]
- KafkaMicroBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs,
- p.failOnDataLoss, p.reuseKafkaConsumer)
- }
-}
-
-/** A [[PartitionReader]] for reading Kafka data in a micro-batch streaming query. */
-private[kafka010] case class KafkaMicroBatchPartitionReader(
- offsetRange: KafkaOffsetRange,
- executorKafkaParams: ju.Map[String, Object],
- pollTimeoutMs: Long,
- failOnDataLoss: Boolean,
- reuseKafkaConsumer: Boolean) extends PartitionReader[InternalRow] with Logging {
-
- private val consumer = KafkaDataConsumer.acquire(
- offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer)
-
- private val rangeToRead = resolveRange(offsetRange)
- private val converter = new KafkaRecordToUnsafeRowConverter
-
- private var nextOffset = rangeToRead.fromOffset
- private var nextRow: UnsafeRow = _
-
- override def next(): Boolean = {
- if (nextOffset < rangeToRead.untilOffset) {
- val record = consumer.get(nextOffset, rangeToRead.untilOffset, pollTimeoutMs, failOnDataLoss)
- if (record != null) {
- nextRow = converter.toUnsafeRow(record)
- nextOffset = record.offset + 1
- true
- } else {
- false
- }
- } else {
- false
- }
- }
-
- override def get(): UnsafeRow = {
- assert(nextRow != null)
- nextRow
- }
-
- override def close(): Unit = {
- consumer.release()
- }
-
- private def resolveRange(range: KafkaOffsetRange): KafkaOffsetRange = {
- if (range.fromOffset < 0 || range.untilOffset < 0) {
- // Late bind the offset range
- val availableOffsetRange = consumer.getAvailableOffsetRange()
- val fromOffset = if (range.fromOffset < 0) {
- assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST,
- s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}")
- availableOffsetRange.earliest
- } else {
- range.fromOffset
- }
- val untilOffset = if (range.untilOffset < 0) {
- assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST,
- s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}")
- availableOffsetRange.latest
- } else {
- range.untilOffset
- }
- KafkaOffsetRange(range.topicPartition, fromOffset, untilOffset, None)
- } else {
- range
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
new file mode 100644
index 0000000..8cc989f
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala
@@ -0,0 +1,382 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.kafka010
+
+import java.{util => ju}
+import java.io._
+import java.nio.charset.StandardCharsets
+
+import scala.collection.JavaConverters._
+
+import org.apache.commons.io.IOUtils
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.ExecutorCacheTaskLocation
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset}
+import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
+import org.apache.spark.sql.sources.v2.DataSourceOptions
+import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
+import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.UninterruptibleThread
+
+/**
+ * A [[MicroBatchReader]] that reads data from Kafka.
+ *
+ * The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains
+ * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For
+ * example if the last record in a Kafka topic "t", partition 2 is offset 5, then
+ * KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent
+ * with the semantics of `KafkaConsumer.position()`.
+ *
+ * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user
+ * must make sure all messages in a topic have been processed when deleting a topic.
+ *
+ * There is a known issue caused by KAFKA-1894: the query using Kafka maybe cannot be stopped.
+ * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers
+ * and not use wrong broker addresses.
+ */
+private[kafka010] class KafkaMicroBatchReader(
+ kafkaOffsetReader: KafkaOffsetReader,
+ executorKafkaParams: ju.Map[String, Object],
+ options: DataSourceOptions,
+ metadataPath: String,
+ startingOffsets: KafkaOffsetRangeLimit,
+ failOnDataLoss: Boolean)
+ extends MicroBatchReader with Logging {
+
+ private var startPartitionOffsets: PartitionOffsetMap = _
+ private var endPartitionOffsets: PartitionOffsetMap = _
+
+ private val pollTimeoutMs = options.getLong(
+ "kafkaConsumer.pollTimeoutMs",
+ SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L)
+
+ private val maxOffsetsPerTrigger =
+ Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)
+
+ private val rangeCalculator = KafkaOffsetRangeCalculator(options)
+ /**
+ * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only
+ * called in StreamExecutionThread. Otherwise, interrupting a thread while running
+ * `KafkaConsumer.poll` may hang forever (KAFKA-1894).
+ */
+ private lazy val initialPartitionOffsets = getOrCreateInitialPartitionOffsets()
+
+ override def setOffsetRange(start: ju.Optional[Offset], end: ju.Optional[Offset]): Unit = {
+ // Make sure initialPartitionOffsets is initialized
+ initialPartitionOffsets
+
+ startPartitionOffsets = Option(start.orElse(null))
+ .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets)
+ .getOrElse(initialPartitionOffsets)
+
+ endPartitionOffsets = Option(end.orElse(null))
+ .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets)
+ .getOrElse {
+ val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
+ maxOffsetsPerTrigger.map { maxOffsets =>
+ rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets)
+ }.getOrElse {
+ latestPartitionOffsets
+ }
+ }
+ }
+
+ override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
+ // Find the new partitions, and get their earliest offsets
+ val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet)
+ val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq)
+ if (newPartitionInitialOffsets.keySet != newPartitions) {
+ // We cannot get from offsets for some partitions. It means they got deleted.
+ val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet)
+ reportDataLoss(
+ s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed")
+ }
+ logInfo(s"Partitions added: $newPartitionInitialOffsets")
+ newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) =>
+ reportDataLoss(
+ s"Added partition $p starts from $o instead of 0. Some data may have been missed")
+ }
+
+ // Find deleted partitions, and report data loss if required
+ val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet)
+ if (deletedPartitions.nonEmpty) {
+ reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed")
+ }
+
+ // Use the end partitions to calculate offset ranges to ignore partitions that have
+ // been deleted
+ val topicPartitions = endPartitionOffsets.keySet.filter { tp =>
+ // Ignore partitions that we don't know the from offsets.
+ newPartitionInitialOffsets.contains(tp) || startPartitionOffsets.contains(tp)
+ }.toSeq
+ logDebug("TopicPartitions: " + topicPartitions.mkString(", "))
+
+ // Calculate offset ranges
+ val offsetRanges = rangeCalculator.getRanges(
+ fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets,
+ untilOffsets = endPartitionOffsets,
+ executorLocations = getSortedExecutorList())
+
+ // Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions,
+ // that is, concurrent tasks will not read the same TopicPartitions.
+ val reuseKafkaConsumer = offsetRanges.map(_.topicPartition).toSet.size == offsetRanges.size
+
+ // Generate factories based on the offset ranges
+ offsetRanges.map { range =>
+ new KafkaMicroBatchInputPartition(
+ range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer
+ ): InputPartition[InternalRow]
+ }.asJava
+ }
+
+ override def getStartOffset: Offset = {
+ KafkaSourceOffset(startPartitionOffsets)
+ }
+
+ override def getEndOffset: Offset = {
+ KafkaSourceOffset(endPartitionOffsets)
+ }
+
+ override def deserializeOffset(json: String): Offset = {
+ KafkaSourceOffset(JsonUtils.partitionOffsets(json))
+ }
+
+ override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema
+
+ override def commit(end: Offset): Unit = {}
+
+ override def stop(): Unit = {
+ kafkaOffsetReader.close()
+ }
+
+ override def toString(): String = s"KafkaV2[$kafkaOffsetReader]"
+
+ /**
+ * Read initial partition offsets from the checkpoint, or decide the offsets and write them to
+ * the checkpoint.
+ */
+ private def getOrCreateInitialPartitionOffsets(): PartitionOffsetMap = {
+ // Make sure that `KafkaConsumer.poll` is only called in StreamExecutionThread.
+ // Otherwise, interrupting a thread while running `KafkaConsumer.poll` may hang forever
+ // (KAFKA-1894).
+ assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
+
+ // SparkSession is required for getting Hadoop configuration for writing to checkpoints
+ assert(SparkSession.getActiveSession.nonEmpty)
+
+ val metadataLog =
+ new KafkaSourceInitialOffsetWriter(SparkSession.getActiveSession.get, metadataPath)
+ metadataLog.get(0).getOrElse {
+ val offsets = startingOffsets match {
+ case EarliestOffsetRangeLimit =>
+ KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets())
+ case LatestOffsetRangeLimit =>
+ KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets())
+ case SpecificOffsetRangeLimit(p) =>
+ kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss)
+ }
+ metadataLog.add(0, offsets)
+ logInfo(s"Initial offsets: $offsets")
+ offsets
+ }.partitionToOffsets
+ }
+
+ /** Proportionally distribute limit number of offsets among topicpartitions */
+ private def rateLimit(
+ limit: Long,
+ from: PartitionOffsetMap,
+ until: PartitionOffsetMap): PartitionOffsetMap = {
+ val fromNew = kafkaOffsetReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq)
+ val sizes = until.flatMap {
+ case (tp, end) =>
+ // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it
+ from.get(tp).orElse(fromNew.get(tp)).flatMap { begin =>
+ val size = end - begin
+ logDebug(s"rateLimit $tp size is $size")
+ if (size > 0) Some(tp -> size) else None
+ }
+ }
+ val total = sizes.values.sum.toDouble
+ if (total < 1) {
+ until
+ } else {
+ until.map {
+ case (tp, end) =>
+ tp -> sizes.get(tp).map { size =>
+ val begin = from.get(tp).getOrElse(fromNew(tp))
+ val prorate = limit * (size / total)
+ // Don't completely starve small topicpartitions
+ val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong
+ // Paranoia, make sure not to return an offset that's past end
+ Math.min(end, off)
+ }.getOrElse(end)
+ }
+ }
+ }
+
+ private def getSortedExecutorList(): Array[String] = {
+
+ def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = {
+ if (a.host == b.host) {
+ a.executorId > b.executorId
+ } else {
+ a.host > b.host
+ }
+ }
+
+ val bm = SparkEnv.get.blockManager
+ bm.master.getPeers(bm.blockManagerId).toArray
+ .map(x => ExecutorCacheTaskLocation(x.host, x.executorId))
+ .sortWith(compare)
+ .map(_.toString)
+ }
+
+ /**
+ * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`.
+ * Otherwise, just log a warning.
+ */
+ private def reportDataLoss(message: String): Unit = {
+ if (failOnDataLoss) {
+ throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE")
+ } else {
+ logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE")
+ }
+ }
+
+ /** A version of [[HDFSMetadataLog]] specialized for saving the initial offsets. */
+ class KafkaSourceInitialOffsetWriter(sparkSession: SparkSession, metadataPath: String)
+ extends HDFSMetadataLog[KafkaSourceOffset](sparkSession, metadataPath) {
+
+ val VERSION = 1
+
+ override def serialize(metadata: KafkaSourceOffset, out: OutputStream): Unit = {
+ out.write(0) // A zero byte is written to support Spark 2.1.0 (SPARK-19517)
+ val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
+ writer.write("v" + VERSION + "\n")
+ writer.write(metadata.json)
+ writer.flush
+ }
+
+ override def deserialize(in: InputStream): KafkaSourceOffset = {
+ in.read() // A zero byte is read to support Spark 2.1.0 (SPARK-19517)
+ val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8))
+ // HDFSMetadataLog guarantees that it never creates a partial file.
+ assert(content.length != 0)
+ if (content(0) == 'v') {
+ val indexOfNewLine = content.indexOf("\n")
+ if (indexOfNewLine > 0) {
+ val version = parseVersion(content.substring(0, indexOfNewLine), VERSION)
+ KafkaSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1)))
+ } else {
+ throw new IllegalStateException(
+ s"Log file was malformed: failed to detect the log file version line.")
+ }
+ } else {
+ // The log was generated by Spark 2.1.0
+ KafkaSourceOffset(SerializedOffset(content))
+ }
+ }
+ }
+}
+
+/** A [[InputPartition]] for reading Kafka data in a micro-batch streaming query. */
+private[kafka010] case class KafkaMicroBatchInputPartition(
+ offsetRange: KafkaOffsetRange,
+ executorKafkaParams: ju.Map[String, Object],
+ pollTimeoutMs: Long,
+ failOnDataLoss: Boolean,
+ reuseKafkaConsumer: Boolean) extends InputPartition[InternalRow] {
+
+ override def preferredLocations(): Array[String] = offsetRange.preferredLoc.toArray
+
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new KafkaMicroBatchInputPartitionReader(offsetRange, executorKafkaParams, pollTimeoutMs,
+ failOnDataLoss, reuseKafkaConsumer)
+}
+
+/** A [[InputPartitionReader]] for reading Kafka data in a micro-batch streaming query. */
+private[kafka010] case class KafkaMicroBatchInputPartitionReader(
+ offsetRange: KafkaOffsetRange,
+ executorKafkaParams: ju.Map[String, Object],
+ pollTimeoutMs: Long,
+ failOnDataLoss: Boolean,
+ reuseKafkaConsumer: Boolean) extends InputPartitionReader[InternalRow] with Logging {
+
+ private val consumer = KafkaDataConsumer.acquire(
+ offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer)
+
+ private val rangeToRead = resolveRange(offsetRange)
+ private val converter = new KafkaRecordToUnsafeRowConverter
+
+ private var nextOffset = rangeToRead.fromOffset
+ private var nextRow: UnsafeRow = _
+
+ override def next(): Boolean = {
+ if (nextOffset < rangeToRead.untilOffset) {
+ val record = consumer.get(nextOffset, rangeToRead.untilOffset, pollTimeoutMs, failOnDataLoss)
+ if (record != null) {
+ nextRow = converter.toUnsafeRow(record)
+ nextOffset = record.offset + 1
+ true
+ } else {
+ false
+ }
+ } else {
+ false
+ }
+ }
+
+ override def get(): UnsafeRow = {
+ assert(nextRow != null)
+ nextRow
+ }
+
+ override def close(): Unit = {
+ consumer.release()
+ }
+
+ private def resolveRange(range: KafkaOffsetRange): KafkaOffsetRange = {
+ if (range.fromOffset < 0 || range.untilOffset < 0) {
+ // Late bind the offset range
+ val availableOffsetRange = consumer.getAvailableOffsetRange()
+ val fromOffset = if (range.fromOffset < 0) {
+ assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST,
+ s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}")
+ availableOffsetRange.earliest
+ } else {
+ range.fromOffset
+ }
+ val untilOffset = if (range.untilOffset < 0) {
+ assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST,
+ s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}")
+ availableOffsetRange.latest
+ } else {
+ range.untilOffset
+ }
+ KafkaOffsetRange(range.topicPartition, fromOffset, untilOffset, None)
+ } else {
+ range
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
index 28c9853..d225c1e 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
@@ -30,8 +30,9 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext}
import org.apache.spark.sql.execution.streaming.{Sink, Source}
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.sources.v2._
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
@@ -45,9 +46,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
with StreamSinkProvider
with RelationProvider
with CreatableRelationProvider
- with StreamingWriteSupportProvider
- with ContinuousReadSupportProvider
- with MicroBatchReadSupportProvider
+ with StreamWriteSupport
+ with ContinuousReadSupport
+ with MicroBatchReadSupport
with Logging {
import KafkaSourceProvider._
@@ -107,12 +108,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
/**
- * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport]] to read
- * batches of Kafka data in a micro-batch streaming query.
+ * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader]] to read batches
+ * of Kafka data in a micro-batch streaming query.
*/
- override def createMicroBatchReadSupport(
+ override def createMicroBatchReader(
+ schema: Optional[StructType],
metadataPath: String,
- options: DataSourceOptions): KafkaMicroBatchReadSupport = {
+ options: DataSourceOptions): KafkaMicroBatchReader = {
val parameters = options.asMap().asScala.toMap
validateStreamOptions(parameters)
@@ -138,7 +140,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
parameters,
driverGroupIdPrefix = s"$uniqueGroupId-driver")
- new KafkaMicroBatchReadSupport(
+ new KafkaMicroBatchReader(
kafkaOffsetReader,
kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId),
options,
@@ -148,12 +150,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
/**
- * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport]] to read
+ * Creates a [[ContinuousInputPartitionReader]] to read
* Kafka data in a continuous streaming query.
*/
- override def createContinuousReadSupport(
+ override def createContinuousReader(
+ schema: Optional[StructType],
metadataPath: String,
- options: DataSourceOptions): KafkaContinuousReadSupport = {
+ options: DataSourceOptions): KafkaContinuousReader = {
val parameters = options.asMap().asScala.toMap
validateStreamOptions(parameters)
// Each running query should use its own group id. Otherwise, the query may be only assigned
@@ -178,7 +181,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
parameters,
driverGroupIdPrefix = s"$uniqueGroupId-driver")
- new KafkaContinuousReadSupport(
+ new KafkaContinuousReader(
kafkaOffsetReader,
kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId),
parameters,
@@ -267,11 +270,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
}
- override def createStreamingWriteSupport(
+ override def createStreamWriter(
queryId: String,
schema: StructType,
mode: OutputMode,
- options: DataSourceOptions): StreamingWriteSupport = {
+ options: DataSourceOptions): StreamWriter = {
import scala.collection.JavaConverters._
val spark = SparkSession.getActiveSession.get
@@ -282,7 +285,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
KafkaWriter.validateQuery(
schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic)
- new KafkaStreamingWriteSupport(topic, producerParams, schema)
+ new KafkaStreamWriter(topic, producerParams, schema)
}
private def strategy(caseInsensitiveParams: Map[String, String]) =
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
new file mode 100644
index 0000000..97c577d
--- /dev/null
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.kafka010
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery
+import org.apache.spark.sql.sources.v2.writer._
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
+import org.apache.spark.sql.types.StructType
+
+/**
+ * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we
+ * don't need to really send one.
+ */
+case object KafkaWriterCommitMessage extends WriterCommitMessage
+
+/**
+ * A [[StreamWriter]] for Kafka writing. Responsible for generating the writer factory.
+ *
+ * @param topic The topic this writer is responsible for. If None, topic will be inferred from
+ * a `topic` field in the incoming data.
+ * @param producerParams Parameters for Kafka producers in each task.
+ * @param schema The schema of the input data.
+ */
+class KafkaStreamWriter(
+ topic: Option[String], producerParams: Map[String, String], schema: StructType)
+ extends StreamWriter {
+
+ validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic)
+
+ override def createWriterFactory(): KafkaStreamWriterFactory =
+ KafkaStreamWriterFactory(topic, producerParams, schema)
+
+ override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+ override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+}
+
+/**
+ * A [[StreamingDataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to
+ * generate the per-task data writers.
+ * @param topic The topic that should be written to. If None, topic will be inferred from
+ * a `topic` field in the incoming data.
+ * @param producerParams Parameters for Kafka producers in each task.
+ * @param schema The schema of the input data.
+ */
+case class KafkaStreamWriterFactory(
+ topic: Option[String], producerParams: Map[String, String], schema: StructType)
+ extends DataWriterFactory[InternalRow] {
+
+ override def createDataWriter(
+ partitionId: Int,
+ taskId: Long,
+ epochId: Long): DataWriter[InternalRow] = {
+ new KafkaStreamDataWriter(topic, producerParams, schema.toAttributes)
+ }
+}
+
+/**
+ * A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to
+ * process incoming rows.
+ *
+ * @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred
+ * from a `topic` field in the incoming data.
+ * @param producerParams Parameters to use for the Kafka producer.
+ * @param inputSchema The attributes in the input data.
+ */
+class KafkaStreamDataWriter(
+ targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute])
+ extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] {
+ import scala.collection.JavaConverters._
+
+ private lazy val producer = CachedKafkaProducer.getOrCreate(
+ new java.util.HashMap[String, Object](producerParams.asJava))
+
+ def write(row: InternalRow): Unit = {
+ checkForErrors()
+ sendRow(row, producer)
+ }
+
+ def commit(): WriterCommitMessage = {
+ // Send is asynchronous, but we can't commit until all rows are actually in Kafka.
+ // This requires flushing and then checking that no callbacks produced errors.
+ // We also check for errors before to fail as soon as possible - the check is cheap.
+ checkForErrors()
+ producer.flush()
+ checkForErrors()
+ KafkaWriterCommitMessage
+ }
+
+ def abort(): Unit = {}
+
+ def close(): Unit = {
+ checkForErrors()
+ if (producer != null) {
+ producer.flush()
+ checkForErrors()
+ CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava))
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala
deleted file mode 100644
index 927c56d..0000000
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.kafka010
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery
-import org.apache.spark.sql.sources.v2.writer._
-import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport}
-import org.apache.spark.sql.types.StructType
-
-/**
- * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we
- * don't need to really send one.
- */
-case object KafkaWriterCommitMessage extends WriterCommitMessage
-
-/**
- * A [[StreamingWriteSupport]] for Kafka writing. Responsible for generating the writer factory.
- *
- * @param topic The topic this writer is responsible for. If None, topic will be inferred from
- * a `topic` field in the incoming data.
- * @param producerParams Parameters for Kafka producers in each task.
- * @param schema The schema of the input data.
- */
-class KafkaStreamingWriteSupport(
- topic: Option[String], producerParams: Map[String, String], schema: StructType)
- extends StreamingWriteSupport {
-
- validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic)
-
- override def createStreamingWriterFactory(): KafkaStreamWriterFactory =
- KafkaStreamWriterFactory(topic, producerParams, schema)
-
- override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
- override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
-}
-
-/**
- * A [[StreamingDataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to
- * generate the per-task data writers.
- * @param topic The topic that should be written to. If None, topic will be inferred from
- * a `topic` field in the incoming data.
- * @param producerParams Parameters for Kafka producers in each task.
- * @param schema The schema of the input data.
- */
-case class KafkaStreamWriterFactory(
- topic: Option[String], producerParams: Map[String, String], schema: StructType)
- extends StreamingDataWriterFactory {
-
- override def createWriter(
- partitionId: Int,
- taskId: Long,
- epochId: Long): DataWriter[InternalRow] = {
- new KafkaStreamDataWriter(topic, producerParams, schema.toAttributes)
- }
-}
-
-/**
- * A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to
- * process incoming rows.
- *
- * @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred
- * from a `topic` field in the incoming data.
- * @param producerParams Parameters to use for the Kafka producer.
- * @param inputSchema The attributes in the input data.
- */
-class KafkaStreamDataWriter(
- targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute])
- extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] {
- import scala.collection.JavaConverters._
-
- private lazy val producer = CachedKafkaProducer.getOrCreate(
- new java.util.HashMap[String, Object](producerParams.asJava))
-
- def write(row: InternalRow): Unit = {
- checkForErrors()
- sendRow(row, producer)
- }
-
- def commit(): WriterCommitMessage = {
- // Send is asynchronous, but we can't commit until all rows are actually in Kafka.
- // This requires flushing and then checking that no callbacks produced errors.
- // We also check for errors before to fail as soon as possible - the check is cheap.
- checkForErrors()
- producer.flush()
- checkForErrors()
- KafkaWriterCommitMessage
- }
-
- def abort(): Unit = {}
-
- def close(): Unit = {
- checkForErrors()
- if (producer != null) {
- producer.flush()
- checkForErrors()
- CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava))
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
index af51021..a0e5818 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.spark.sql.Dataset
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec
+import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
import org.apache.spark.sql.streaming.Trigger
@@ -207,13 +207,11 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest {
testUtils.createTopic(topic2, partitions = 5)
eventually(timeout(streamingTimeout)) {
assert(
- query.lastExecution.executedPlan.collectFirst {
- case scan: DataSourceV2ScanExec
- if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] =>
- scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig]
- }.exists { config =>
+ query.lastExecution.logical.collectFirst {
+ case StreamingDataSourceV2Relation(_, _, _, r: KafkaContinuousReader) => r
+ }.exists { r =>
// Ensure the new topic is present and the old topic is gone.
- config.knownPartitions.exists(_.topic == topic2)
+ r.knownPartitions.exists(_.topic == topic2)
},
s"query never reconfigured to new topic $topic2")
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala
index fa6bdc2..fa1468a 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger
import org.apache.spark.SparkContext
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart}
-import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec
+import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
import org.apache.spark.sql.execution.streaming.StreamExecution
import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
import org.apache.spark.sql.streaming.Trigger
@@ -46,10 +46,8 @@ trait KafkaContinuousTest extends KafkaSourceTest {
testUtils.addPartitions(topic, newCount)
eventually(timeout(streamingTimeout)) {
assert(
- query.lastExecution.executedPlan.collectFirst {
- case scan: DataSourceV2ScanExec
- if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] =>
- scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig]
+ query.lastExecution.logical.collectFirst {
+ case StreamingDataSourceV2Relation(_, _, _, r: KafkaContinuousReader) => r
}.exists(_.knownPartitions.size == newCount),
s"query never reconfigured to $newCount partitions")
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
----------------------------------------------------------------------
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 8e246db..65615fd 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
import java.io._
import java.nio.charset.StandardCharsets.UTF_8
import java.nio.file.{Files, Paths}
-import java.util.Locale
+import java.util.{Locale, Optional}
import java.util.concurrent.ConcurrentLinkedQueue
import java.util.concurrent.atomic.AtomicInteger
@@ -28,7 +28,7 @@ import scala.collection.JavaConverters._
import scala.io.Source
import scala.util.Random
-import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata}
+import org.apache.kafka.clients.producer.{ProducerRecord, RecordMetadata}
import org.apache.kafka.common.TopicPartition
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.time.SpanSugar._
@@ -40,9 +40,11 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution
import org.apache.spark.sql.functions.{count, window}
import org.apache.spark.sql.kafka010.KafkaSourceProvider._
import org.apache.spark.sql.sources.v2.DataSourceOptions
+import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2}
import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest}
import org.apache.spark.sql.streaming.util.StreamManualClock
import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest {
@@ -112,16 +114,14 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with Kaf
query.nonEmpty,
"Cannot add data when there is no query for finding the active kafka source")
- val sources: Seq[BaseStreamingSource] = {
+ val sources = {
query.get.logicalPlan.collect {
case StreamingExecutionRelation(source: KafkaSource, _) => source
- case StreamingExecutionRelation(source: KafkaMicroBatchReadSupport, _) => source
+ case StreamingExecutionRelation(source: KafkaMicroBatchReader, _) => source
} ++ (query.get.lastExecution match {
case null => Seq()
case e => e.logical.collect {
- case r: StreamingDataSourceV2Relation
- if r.readSupport.isInstanceOf[KafkaContinuousReadSupport] =>
- r.readSupport.asInstanceOf[KafkaContinuousReadSupport]
+ case StreamingDataSourceV2Relation(_, _, _, reader: KafkaContinuousReader) => reader
}
})
}.distinct
@@ -905,7 +905,7 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase {
makeSureGetOffsetCalled,
AssertOnQuery { query =>
query.logicalPlan.collect {
- case StreamingExecutionRelation(_: KafkaMicroBatchReadSupport, _) => true
+ case StreamingExecutionRelation(_: KafkaMicroBatchReader, _) => true
}.nonEmpty
}
)
@@ -930,16 +930,17 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase {
"kafka.bootstrap.servers" -> testUtils.brokerAddress,
"subscribe" -> topic
) ++ Option(minPartitions).map { p => "minPartitions" -> p}
- val readSupport = provider.createMicroBatchReadSupport(
- dir.getAbsolutePath, new DataSourceOptions(options.asJava))
- val config = readSupport.newScanConfigBuilder(
- KafkaSourceOffset(Map(tp -> 0L)),
- KafkaSourceOffset(Map(tp -> 100L))).build()
- val inputPartitions = readSupport.planInputPartitions(config)
+ val reader = provider.createMicroBatchReader(
+ Optional.empty[StructType], dir.getAbsolutePath, new DataSourceOptions(options.asJava))
+ reader.setOffsetRange(
+ Optional.of[OffsetV2](KafkaSourceOffset(Map(tp -> 0L))),
+ Optional.of[OffsetV2](KafkaSourceOffset(Map(tp -> 100L)))
+ )
+ val factories = reader.planInputPartitions().asScala
.map(_.asInstanceOf[KafkaMicroBatchInputPartition])
- withClue(s"minPartitions = $minPartitions generated factories $inputPartitions\n\t") {
- assert(inputPartitions.size == numPartitionsGenerated)
- inputPartitions.foreach { f => assert(f.reuseKafkaConsumer == reusesConsumers) }
+ withClue(s"minPartitions = $minPartitions generated factories $factories\n\t") {
+ assert(factories.size == numPartitionsGenerated)
+ factories.foreach { f => assert(f.reuseKafkaConsumer == reusesConsumers) }
}
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[4/7] spark git commit: [SPARK-24882][SQL] Revert [] improve data
source v2 API from branch 2.4
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
index 2cac865..7a007b6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.execution.streaming
+import java.util.Optional
+
import scala.collection.JavaConverters._
import scala.collection.mutable.{Map => MutableMap}
@@ -26,9 +28,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentBatchTimestamp,
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project}
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2}
-import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport}
-import org.apache.spark.sql.sources.v2._
-import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2}
+import org.apache.spark.sql.execution.streaming.sources.MicroBatchWriter
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, MicroBatchReadSupport, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset => OffsetV2}
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
import org.apache.spark.util.{Clock, Utils}
@@ -49,8 +51,8 @@ class MicroBatchExecution(
@volatile protected var sources: Seq[BaseStreamingSource] = Seq.empty
- private val readSupportToDataSourceMap =
- MutableMap.empty[MicroBatchReadSupport, (DataSourceV2, Map[String, String])]
+ private val readerToDataSourceMap =
+ MutableMap.empty[MicroBatchReader, (DataSourceV2, Map[String, String])]
private val triggerExecutor = trigger match {
case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock)
@@ -89,19 +91,20 @@ class MicroBatchExecution(
StreamingExecutionRelation(source, output)(sparkSession)
})
case s @ StreamingRelationV2(
- dataSourceV2: MicroBatchReadSupportProvider, sourceName, options, output, _) if
+ dataSourceV2: MicroBatchReadSupport, sourceName, options, output, _) if
!disabledSources.contains(dataSourceV2.getClass.getCanonicalName) =>
v2ToExecutionRelationMap.getOrElseUpdate(s, {
// Materialize source to avoid creating it in every batch
val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId"
- val readSupport = dataSourceV2.createMicroBatchReadSupport(
+ val reader = dataSourceV2.createMicroBatchReader(
+ Optional.empty(), // user specified schema
metadataPath,
new DataSourceOptions(options.asJava))
nextSourceId += 1
- readSupportToDataSourceMap(readSupport) = dataSourceV2 -> options
- logInfo(s"Using MicroBatchReadSupport [$readSupport] from " +
+ readerToDataSourceMap(reader) = dataSourceV2 -> options
+ logInfo(s"Using MicroBatchReader [$reader] from " +
s"DataSourceV2 named '$sourceName' [$dataSourceV2]")
- StreamingExecutionRelation(readSupport, output)(sparkSession)
+ StreamingExecutionRelation(reader, output)(sparkSession)
})
case s @ StreamingRelationV2(dataSourceV2, sourceName, _, output, v1Relation) =>
v2ToExecutionRelationMap.getOrElseUpdate(s, {
@@ -341,19 +344,19 @@ class MicroBatchExecution(
reportTimeTaken("getOffset") {
(s, s.getOffset)
}
- case s: RateControlMicroBatchReadSupport =>
- updateStatusMessage(s"Getting offsets from $s")
- reportTimeTaken("latestOffset") {
- val startOffset = availableOffsets
- .get(s).map(off => s.deserializeOffset(off.json))
- .getOrElse(s.initialOffset())
- (s, Option(s.latestOffset(startOffset)))
- }
- case s: MicroBatchReadSupport =>
+ case s: MicroBatchReader =>
updateStatusMessage(s"Getting offsets from $s")
- reportTimeTaken("latestOffset") {
- (s, Option(s.latestOffset()))
+ reportTimeTaken("setOffsetRange") {
+ // Once v1 streaming source execution is gone, we can refactor this away.
+ // For now, we set the range here to get the source to infer the available end offset,
+ // get that offset, and then set the range again when we later execute.
+ s.setOffsetRange(
+ toJava(availableOffsets.get(s).map(off => s.deserializeOffset(off.json))),
+ Optional.empty())
}
+
+ val currentOffset = reportTimeTaken("getEndOffset") { s.getEndOffset() }
+ (s, Option(currentOffset))
}.toMap
availableOffsets ++= latestOffsets.filter { case (_, o) => o.nonEmpty }.mapValues(_.get)
@@ -393,8 +396,8 @@ class MicroBatchExecution(
if (prevBatchOff.isDefined) {
prevBatchOff.get.toStreamProgress(sources).foreach {
case (src: Source, off) => src.commit(off)
- case (readSupport: MicroBatchReadSupport, off) =>
- readSupport.commit(readSupport.deserializeOffset(off.json))
+ case (reader: MicroBatchReader, off) =>
+ reader.commit(reader.deserializeOffset(off.json))
case (src, _) =>
throw new IllegalArgumentException(
s"Unknown source is found at constructNextBatch: $src")
@@ -438,34 +441,30 @@ class MicroBatchExecution(
s"${batch.queryExecution.logical}")
logDebug(s"Retrieving data from $source: $current -> $available")
Some(source -> batch.logicalPlan)
-
- // TODO(cloud-fan): for data source v2, the new batch is just a new `ScanConfigBuilder`, but
- // to be compatible with streaming source v1, we return a logical plan as a new batch here.
- case (readSupport: MicroBatchReadSupport, available)
- if committedOffsets.get(readSupport).map(_ != available).getOrElse(true) =>
- val current = committedOffsets.get(readSupport).map {
- off => readSupport.deserializeOffset(off.json)
- }
- val endOffset: OffsetV2 = available match {
- case v1: SerializedOffset => readSupport.deserializeOffset(v1.json)
+ case (reader: MicroBatchReader, available)
+ if committedOffsets.get(reader).map(_ != available).getOrElse(true) =>
+ val current = committedOffsets.get(reader).map(off => reader.deserializeOffset(off.json))
+ val availableV2: OffsetV2 = available match {
+ case v1: SerializedOffset => reader.deserializeOffset(v1.json)
case v2: OffsetV2 => v2
}
- val startOffset = current.getOrElse(readSupport.initialOffset)
- val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset, endOffset)
- logDebug(s"Retrieving data from $readSupport: $current -> $endOffset")
+ reader.setOffsetRange(
+ toJava(current),
+ Optional.of(availableV2))
+ logDebug(s"Retrieving data from $reader: $current -> $availableV2")
- val (source, options) = readSupport match {
+ val (source, options) = reader match {
// `MemoryStream` is special. It's for test only and doesn't have a `DataSourceV2`
// implementation. We provide a fake one here for explain.
case _: MemoryStream[_] => MemoryStreamDataSource -> Map.empty[String, String]
// Provide a fake value here just in case something went wrong, e.g. the reader gives
// a wrong `equals` implementation.
- case _ => readSupportToDataSourceMap.getOrElse(readSupport, {
+ case _ => readerToDataSourceMap.getOrElse(reader, {
FakeDataSourceV2 -> Map.empty[String, String]
})
}
- Some(readSupport -> StreamingDataSourceV2Relation(
- readSupport.fullSchema().toAttributes, source, options, readSupport, scanConfigBuilder))
+ Some(reader -> StreamingDataSourceV2Relation(
+ reader.readSchema().toAttributes, source, options, reader))
case _ => None
}
}
@@ -499,13 +498,13 @@ class MicroBatchExecution(
val triggerLogicalPlan = sink match {
case _: Sink => newAttributePlan
- case s: StreamingWriteSupportProvider =>
- val writer = s.createStreamingWriteSupport(
+ case s: StreamWriteSupport =>
+ val writer = s.createStreamWriter(
s"$runId",
newAttributePlan.schema,
outputMode,
new DataSourceOptions(extraOptions.asJava))
- WriteToDataSourceV2(new MicroBatchWritSupport(currentBatchId, writer), newAttributePlan)
+ WriteToDataSourceV2(new MicroBatchWriter(currentBatchId, writer), newAttributePlan)
case _ => throw new IllegalArgumentException(s"unknown sink type for $sink")
}
@@ -533,7 +532,7 @@ class MicroBatchExecution(
SQLExecution.withNewExecutionId(sparkSessionToRunBatch, lastExecution) {
sink match {
case s: Sink => s.addBatch(currentBatchId, nextBatch)
- case _: StreamingWriteSupportProvider =>
+ case _: StreamWriteSupport =>
// This doesn't accumulate any data - it just forces execution of the microbatch writer.
nextBatch.collect()
}
@@ -557,6 +556,10 @@ class MicroBatchExecution(
awaitProgressLock.unlock()
}
}
+
+ private def toJava(scalaOption: Option[OffsetV2]): Optional[OffsetV2] = {
+ Optional.ofNullable(scalaOption.orNull)
+ }
}
object MicroBatchExecution {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
index d4b5065..6a380ab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalP
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec
-import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader
import org.apache.spark.sql.streaming._
import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent
import org.apache.spark.util.Clock
@@ -251,7 +251,7 @@ trait ProgressReporter extends Logging {
// Check whether the streaming query's logical plan has only V2 data sources
val allStreamingLeaves =
logicalPlan.collect { case s: StreamingExecutionRelation => s }
- allStreamingLeaves.forall { _.source.isInstanceOf[MicroBatchReadSupport] }
+ allStreamingLeaves.forall { _.source.isInstanceOf[MicroBatchReader] }
}
if (onlyDataSourceV2Sources) {
@@ -278,7 +278,7 @@ trait ProgressReporter extends Logging {
new IdentityHashMap[DataSourceV2ScanExec, DataSourceV2ScanExec]()
lastExecution.executedPlan.collectLeaves().foreach {
- case s: DataSourceV2ScanExec if s.readSupport.isInstanceOf[BaseStreamingSource] =>
+ case s: DataSourceV2ScanExec if s.reader.isInstanceOf[BaseStreamingSource] =>
uniqueStreamingExecLeavesMap.put(s, s)
case _ =>
}
@@ -286,7 +286,7 @@ trait ProgressReporter extends Logging {
val sourceToInputRowsTuples =
uniqueStreamingExecLeavesMap.values.asScala.map { execLeaf =>
val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L)
- val source = execLeaf.readSupport.asInstanceOf[BaseStreamingSource]
+ val source = execLeaf.reader.asInstanceOf[BaseStreamingSource]
source -> numRows
}.toSeq
logDebug("Source -> # input rows\n\t" + sourceToInputRowsTuples.mkString("\n\t"))
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala
deleted file mode 100644
index 1be0716..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming
-
-import org.apache.spark.sql.sources.v2.reader.{ScanConfig, ScanConfigBuilder}
-import org.apache.spark.sql.types.StructType
-
-/**
- * A very simple [[ScanConfigBuilder]] implementation that creates a simple [[ScanConfig]] to
- * carry schema and offsets for streaming data sources.
- */
-class SimpleStreamingScanConfigBuilder(
- schema: StructType,
- start: Offset,
- end: Option[Offset] = None)
- extends ScanConfigBuilder {
-
- override def build(): ScanConfig = SimpleStreamingScanConfig(schema, start, end)
-}
-
-case class SimpleStreamingScanConfig(
- readSchema: StructType,
- start: Offset,
- end: Option[Offset])
- extends ScanConfig
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
index 4b696df..24195b5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.execution.LeafExecNode
import org.apache.spark.sql.execution.datasources.DataSource
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceV2}
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2}
object StreamingRelation {
def apply(dataSource: DataSource): StreamingRelation = {
@@ -83,7 +83,7 @@ case class StreamingExecutionRelation(
// We have to pack in the V1 data source as a shim, for the case when a source implements
// continuous processing (which is always V2) but only has V1 microbatch support. We don't
-// know at read time whether the query is continuous or not, so we need to be able to
+// know at read time whether the query is conntinuous or not, so we need to be able to
// swap a V1 relation back in.
/**
* Used to link a [[DataSourceV2]] into a streaming
@@ -113,7 +113,7 @@ case class StreamingRelationV2(
* Used to link a [[DataSourceV2]] into a continuous processing execution.
*/
case class ContinuousExecutionRelation(
- source: ContinuousReadSupportProvider,
+ source: ContinuousReadSupport,
extraOptions: Map[String, String],
output: Seq[Attribute])(session: SparkSession)
extends LeafNode with MultiInstanceRelation {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
index 9c5c16f..cfba100 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql.execution.streaming
import org.apache.spark.sql._
-import org.apache.spark.sql.execution.streaming.sources.ConsoleWriteSupport
+import org.apache.spark.sql.execution.streaming.sources.ConsoleWriter
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister}
-import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider}
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
@@ -31,16 +31,16 @@ case class ConsoleRelation(override val sqlContext: SQLContext, data: DataFrame)
}
class ConsoleSinkProvider extends DataSourceV2
- with StreamingWriteSupportProvider
+ with StreamWriteSupport
with DataSourceRegister
with CreatableRelationProvider {
- override def createStreamingWriteSupport(
+ override def createStreamWriter(
queryId: String,
schema: StructType,
mode: OutputMode,
- options: DataSourceOptions): StreamingWriteSupport = {
- new ConsoleWriteSupport(schema, options)
+ options: DataSourceOptions): StreamWriter = {
+ new ConsoleWriter(schema, options)
}
def createRelation(
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
index b68f67e..554a0b0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala
@@ -21,13 +21,12 @@ import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousPartitionReaderFactory
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader
import org.apache.spark.util.NextIterator
class ContinuousDataSourceRDDPartition(
val index: Int,
- val inputPartition: InputPartition)
+ val inputPartition: InputPartition[InternalRow])
extends Partition with Serializable {
// This is semantically a lazy val - it's initialized once the first time a call to
@@ -50,22 +49,15 @@ class ContinuousDataSourceRDD(
sc: SparkContext,
dataQueueSize: Int,
epochPollIntervalMs: Long,
- private val inputPartitions: Seq[InputPartition],
- schema: StructType,
- partitionReaderFactory: ContinuousPartitionReaderFactory)
+ private val readerInputPartitions: Seq[InputPartition[InternalRow]])
extends RDD[InternalRow](sc, Nil) {
override protected def getPartitions: Array[Partition] = {
- inputPartitions.zipWithIndex.map {
+ readerInputPartitions.zipWithIndex.map {
case (inputPartition, index) => new ContinuousDataSourceRDDPartition(index, inputPartition)
}.toArray
}
- private def castPartition(split: Partition): ContinuousDataSourceRDDPartition = split match {
- case p: ContinuousDataSourceRDDPartition => p
- case _ => throw new SparkException(s"[BUG] Not a ContinuousDataSourceRDDPartition: $split")
- }
-
/**
* Initialize the shared reader for this partition if needed, then read rows from it until
* it returns null to signal the end of the epoch.
@@ -77,12 +69,10 @@ class ContinuousDataSourceRDD(
}
val readerForPartition = {
- val partition = castPartition(split)
+ val partition = split.asInstanceOf[ContinuousDataSourceRDDPartition]
if (partition.queueReader == null) {
- val partitionReader = partitionReaderFactory.createReader(
- partition.inputPartition)
- partition.queueReader = new ContinuousQueuedDataReader(
- partition.index, partitionReader, schema, context, dataQueueSize, epochPollIntervalMs)
+ partition.queueReader =
+ new ContinuousQueuedDataReader(partition, context, dataQueueSize, epochPollIntervalMs)
}
partition.queueReader
@@ -103,6 +93,17 @@ class ContinuousDataSourceRDD(
}
override def getPreferredLocations(split: Partition): Seq[String] = {
- castPartition(split).inputPartition.preferredLocations()
+ split.asInstanceOf[ContinuousDataSourceRDDPartition].inputPartition.preferredLocations()
+ }
+}
+
+object ContinuousDataSourceRDD {
+ private[continuous] def getContinuousReader(
+ reader: InputPartitionReader[InternalRow]): ContinuousInputPartitionReader[_] = {
+ reader match {
+ case r: ContinuousInputPartitionReader[InternalRow] => r
+ case _ =>
+ throw new IllegalStateException(s"Unknown continuous reader type ${reader.getClass}")
+ }
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
index ccca726..f104422 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
@@ -29,12 +29,13 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentDate, CurrentTimestamp}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.SQLExecution
-import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, StreamingDataSourceV2Relation}
+import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2}
import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _}
import org.apache.spark.sql.sources.v2
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider}
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset}
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset}
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
+import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{Clock, Utils}
class ContinuousExecution(
@@ -42,7 +43,7 @@ class ContinuousExecution(
name: String,
checkpointRoot: String,
analyzedPlan: LogicalPlan,
- sink: StreamingWriteSupportProvider,
+ sink: StreamWriteSupport,
trigger: Trigger,
triggerClock: Clock,
outputMode: OutputMode,
@@ -52,7 +53,7 @@ class ContinuousExecution(
sparkSession, name, checkpointRoot, analyzedPlan, sink,
trigger, triggerClock, outputMode, deleteCheckpointOnStop) {
- @volatile protected var continuousSources: Seq[ContinuousReadSupport] = Seq()
+ @volatile protected var continuousSources: Seq[ContinuousReader] = Seq()
override protected def sources: Seq[BaseStreamingSource] = continuousSources
// For use only in test harnesses.
@@ -62,8 +63,7 @@ class ContinuousExecution(
val toExecutionRelationMap = MutableMap[StreamingRelationV2, ContinuousExecutionRelation]()
analyzedPlan.transform {
case r @ StreamingRelationV2(
- source: ContinuousReadSupportProvider, _, extraReaderOptions, output, _) =>
- // TODO: shall we create `ContinuousReadSupport` here instead of each reconfiguration?
+ source: ContinuousReadSupport, _, extraReaderOptions, output, _) =>
toExecutionRelationMap.getOrElseUpdate(r, {
ContinuousExecutionRelation(source, extraReaderOptions, output)(sparkSession)
})
@@ -148,7 +148,8 @@ class ContinuousExecution(
val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId"
nextSourceId += 1
- dataSource.createContinuousReadSupport(
+ dataSource.createContinuousReader(
+ java.util.Optional.empty[StructType](),
metadataPath,
new DataSourceOptions(extraReaderOptions.asJava))
}
@@ -159,9 +160,9 @@ class ContinuousExecution(
var insertedSourceId = 0
val withNewSources = logicalPlan transform {
case ContinuousExecutionRelation(source, options, output) =>
- val readSupport = continuousSources(insertedSourceId)
+ val reader = continuousSources(insertedSourceId)
insertedSourceId += 1
- val newOutput = readSupport.fullSchema().toAttributes
+ val newOutput = reader.readSchema().toAttributes
assert(output.size == newOutput.size,
s"Invalid reader: ${Utils.truncatedString(output, ",")} != " +
@@ -169,10 +170,9 @@ class ContinuousExecution(
replacements ++= output.zip(newOutput)
val loggedOffset = offsets.offsets(0)
- val realOffset = loggedOffset.map(off => readSupport.deserializeOffset(off.json))
- val startOffset = realOffset.getOrElse(readSupport.initialOffset)
- val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset)
- StreamingDataSourceV2Relation(newOutput, source, options, readSupport, scanConfigBuilder)
+ val realOffset = loggedOffset.map(off => reader.deserializeOffset(off.json))
+ reader.setStartOffset(java.util.Optional.ofNullable(realOffset.orNull))
+ StreamingDataSourceV2Relation(newOutput, source, options, reader)
}
// Rewire the plan to use the new attributes that were returned by the source.
@@ -185,13 +185,17 @@ class ContinuousExecution(
"CurrentTimestamp and CurrentDate not yet supported for continuous processing")
}
- val writer = sink.createStreamingWriteSupport(
+ val writer = sink.createStreamWriter(
s"$runId",
triggerLogicalPlan.schema,
outputMode,
new DataSourceOptions(extraOptions.asJava))
val withSink = WriteToContinuousDataSource(writer, triggerLogicalPlan)
+ val reader = withSink.collect {
+ case StreamingDataSourceV2Relation(_, _, _, r: ContinuousReader) => r
+ }.head
+
reportTimeTaken("queryPlanning") {
lastExecution = new IncrementalExecution(
sparkSessionForQuery,
@@ -204,11 +208,6 @@ class ContinuousExecution(
lastExecution.executedPlan // Force the lazy generation of execution plan
}
- val (readSupport, scanConfig) = lastExecution.executedPlan.collect {
- case scan: DataSourceV2ScanExec if scan.readSupport.isInstanceOf[ContinuousReadSupport] =>
- scan.readSupport.asInstanceOf[ContinuousReadSupport] -> scan.scanConfig
- }.head
-
sparkSessionForQuery.sparkContext.setLocalProperty(
StreamExecution.IS_CONTINUOUS_PROCESSING, true.toString)
sparkSessionForQuery.sparkContext.setLocalProperty(
@@ -226,16 +225,14 @@ class ContinuousExecution(
// Use the parent Spark session for the endpoint since it's where this query ID is registered.
val epochEndpoint =
EpochCoordinatorRef.create(
- writer, readSupport, this, epochCoordinatorId, currentBatchId, sparkSession, SparkEnv.get)
+ writer, reader, this, epochCoordinatorId, currentBatchId, sparkSession, SparkEnv.get)
val epochUpdateThread = new Thread(new Runnable {
override def run: Unit = {
try {
triggerExecutor.execute(() => {
startTrigger()
- val shouldReconfigure = readSupport.needsReconfiguration(scanConfig) &&
- state.compareAndSet(ACTIVE, RECONFIGURING)
- if (shouldReconfigure) {
+ if (reader.needsReconfiguration() && state.compareAndSet(ACTIVE, RECONFIGURING)) {
if (queryExecutionThread.isAlive) {
queryExecutionThread.interrupt()
}
@@ -285,12 +282,10 @@ class ContinuousExecution(
* Report ending partition offsets for the given reader at the given epoch.
*/
def addOffset(
- epoch: Long,
- readSupport: ContinuousReadSupport,
- partitionOffsets: Seq[PartitionOffset]): Unit = {
+ epoch: Long, reader: ContinuousReader, partitionOffsets: Seq[PartitionOffset]): Unit = {
assert(continuousSources.length == 1, "only one continuous source supported currently")
- val globalOffset = readSupport.mergeOffsets(partitionOffsets.toArray)
+ val globalOffset = reader.mergeOffsets(partitionOffsets.toArray)
val oldOffset = synchronized {
offsetLog.add(epoch, OffsetSeq.fill(globalOffset))
offsetLog.get(epoch - 1)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
index 65c5fc6..ec1dabd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala
@@ -25,9 +25,8 @@ import scala.util.control.NonFatal
import org.apache.spark.{SparkEnv, SparkException, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, PartitionOffset}
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
+import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset
import org.apache.spark.util.ThreadUtils
/**
@@ -38,14 +37,15 @@ import org.apache.spark.util.ThreadUtils
* offsets across epochs. Each compute() should call the next() method here until null is returned.
*/
class ContinuousQueuedDataReader(
- partitionIndex: Int,
- reader: ContinuousPartitionReader[InternalRow],
- schema: StructType,
+ partition: ContinuousDataSourceRDDPartition,
context: TaskContext,
dataQueueSize: Int,
epochPollIntervalMs: Long) extends Closeable {
+ private val reader = partition.inputPartition.createPartitionReader()
+
// Important sequencing - we must get our starting point before the provider threads start running
- private var currentOffset: PartitionOffset = reader.getOffset
+ private var currentOffset: PartitionOffset =
+ ContinuousDataSourceRDD.getContinuousReader(reader).getOffset
/**
* The record types in the read buffer.
@@ -66,7 +66,7 @@ class ContinuousQueuedDataReader(
epochMarkerExecutor.scheduleWithFixedDelay(
epochMarkerGenerator, 0, epochPollIntervalMs, TimeUnit.MILLISECONDS)
- private val dataReaderThread = new DataReaderThread(schema)
+ private val dataReaderThread = new DataReaderThread
dataReaderThread.setDaemon(true)
dataReaderThread.start()
@@ -113,7 +113,7 @@ class ContinuousQueuedDataReader(
currentEntry match {
case EpochMarker =>
epochCoordEndpoint.send(ReportPartitionOffset(
- partitionIndex, EpochTracker.getCurrentEpoch.get, currentOffset))
+ partition.index, EpochTracker.getCurrentEpoch.get, currentOffset))
null
case ContinuousRow(row, offset) =>
currentOffset = offset
@@ -128,16 +128,16 @@ class ContinuousQueuedDataReader(
/**
* The data component of [[ContinuousQueuedDataReader]]. Pushes (row, offset) to the queue when
- * a new row arrives to the [[ContinuousPartitionReader]].
+ * a new row arrives to the [[InputPartitionReader]].
*/
- class DataReaderThread(schema: StructType) extends Thread(
+ class DataReaderThread extends Thread(
s"continuous-reader--${context.partitionId()}--" +
s"${context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY)}") with Logging {
@volatile private[continuous] var failureReason: Throwable = _
- private val toUnsafe = UnsafeProjection.create(schema)
override def run(): Unit = {
TaskContext.setTaskContext(context)
+ val baseReader = ContinuousDataSourceRDD.getContinuousReader(reader)
try {
while (!shouldStop()) {
if (!reader.next()) {
@@ -149,9 +149,8 @@ class ContinuousQueuedDataReader(
return
}
}
- // `InternalRow#copy` may not be properly implemented, for safety we convert to unsafe row
- // before copy here.
- queue.put(ContinuousRow(toUnsafe(reader.get()).copy(), reader.getOffset))
+
+ queue.put(ContinuousRow(reader.get().copy(), baseReader.getOffset))
}
} catch {
case _: InterruptedException =>
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala
----------------------------------------------------------------------
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 a6cde2b..551e07c 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
@@ -17,22 +17,24 @@
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.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.streaming.{RateStreamOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder, ValueRunTimeMsPair}
+import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair}
import org.apache.spark.sql.execution.streaming.sources.RateStreamProvider
import org.apache.spark.sql.sources.v2.DataSourceOptions
import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming._
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset}
import org.apache.spark.sql.types.StructType
case class RateStreamPartitionOffset(
partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset
-class RateStreamContinuousReadSupport(options: DataSourceOptions) extends ContinuousReadSupport {
+class RateStreamContinuousReader(options: DataSourceOptions) extends ContinuousReader {
implicit val defaultFormats: DefaultFormats = DefaultFormats
val creationTime = System.currentTimeMillis()
@@ -54,18 +56,18 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin
RateStreamOffset(Serialization.read[Map[Int, ValueRunTimeMsPair]](json))
}
- override def fullSchema(): StructType = RateStreamProvider.SCHEMA
+ override def readSchema(): StructType = RateStreamProvider.SCHEMA
- override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start)
- }
+ private var offset: Offset = _
- override def initialOffset: Offset = createInitialOffset(numPartitions, creationTime)
+ override def setStartOffset(offset: java.util.Optional[Offset]): Unit = {
+ this.offset = offset.orElse(createInitialOffset(numPartitions, creationTime))
+ }
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val startOffset = config.asInstanceOf[SimpleStreamingScanConfig].start
+ override def getStartOffset(): Offset = offset
- val partitionStartMap = startOffset match {
+ override def planInputPartitions(): java.util.List[InputPartition[InternalRow]] = {
+ val partitionStartMap = offset match {
case off: RateStreamOffset => off.partitionToValueAndRunTimeMs
case off =>
throw new IllegalArgumentException(
@@ -88,12 +90,8 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin
i,
numPartitions,
perPartitionRate)
- }.toArray
- }
-
- override def createContinuousReaderFactory(
- config: ScanConfig): ContinuousPartitionReaderFactory = {
- RateStreamContinuousReaderFactory
+ .asInstanceOf[InputPartition[InternalRow]]
+ }.asJava
}
override def commit(end: Offset): Unit = {}
@@ -120,23 +118,33 @@ case class RateStreamContinuousInputPartition(
partitionIndex: Int,
increment: Long,
rowsPerSecond: Double)
- extends InputPartition
-
-object RateStreamContinuousReaderFactory extends ContinuousPartitionReaderFactory {
- override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[RateStreamContinuousInputPartition]
- new RateStreamContinuousPartitionReader(
- p.startValue, p.startTimeMs, p.partitionIndex, p.increment, p.rowsPerSecond)
+ extends ContinuousInputPartition[InternalRow] {
+
+ override def createContinuousReader(
+ offset: PartitionOffset): InputPartitionReader[InternalRow] = {
+ val rateStreamOffset = offset.asInstanceOf[RateStreamPartitionOffset]
+ require(rateStreamOffset.partition == partitionIndex,
+ s"Expected partitionIndex: $partitionIndex, but got: ${rateStreamOffset.partition}")
+ new RateStreamContinuousInputPartitionReader(
+ rateStreamOffset.currentValue,
+ rateStreamOffset.currentTimeMs,
+ partitionIndex,
+ increment,
+ rowsPerSecond)
}
+
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new RateStreamContinuousInputPartitionReader(
+ startValue, startTimeMs, partitionIndex, increment, rowsPerSecond)
}
-class RateStreamContinuousPartitionReader(
+class RateStreamContinuousInputPartitionReader(
startValue: Long,
startTimeMs: Long,
partitionIndex: Int,
increment: Long,
rowsPerSecond: Double)
- extends ContinuousPartitionReader[InternalRow] {
+ extends ContinuousInputPartitionReader[InternalRow] {
private var nextReadTime: Long = startTimeMs
private val readTimeIncrement: Long = (1000 / rowsPerSecond).toLong
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
index 28ab244..56bfefd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala
@@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.streaming.continuous
import java.io.{BufferedReader, InputStreamReader, IOException}
import java.net.Socket
import java.sql.Timestamp
-import java.util.Calendar
+import java.util.{Calendar, List => JList}
import javax.annotation.concurrent.GuardedBy
+import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import org.json4s.{DefaultFormats, NoTypeHints}
@@ -33,26 +34,24 @@ import org.apache.spark.internal.Logging
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.streaming.{Offset => _, _}
+import org.apache.spark.sql.execution.streaming.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord}
import org.apache.spark.sql.execution.streaming.sources.TextSocketReader
import org.apache.spark.sql.sources.v2.DataSourceOptions
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming._
+import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset}
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.RpcUtils
/**
- * A ContinuousReadSupport that reads text lines through a TCP socket, designed only for tutorials
- * and debugging. This ContinuousReadSupport will *not* work in production applications due to
- * multiple reasons, including no support for fault recovery.
+ * A ContinuousReader that reads text lines through a TCP socket, designed only for tutorials and
+ * debugging. This ContinuousReader will *not* work in production applications due to multiple
+ * reasons, including no support for fault recovery.
*
* The driver maintains a socket connection to the host-port, keeps the received messages in
* buckets and serves the messages to the executors via a RPC endpoint.
*/
-class TextSocketContinuousReadSupport(options: DataSourceOptions)
- extends ContinuousReadSupport with Logging {
-
+class TextSocketContinuousReader(options: DataSourceOptions) extends ContinuousReader with Logging {
implicit val defaultFormats: DefaultFormats = DefaultFormats
private val host: String = options.get("host").get()
@@ -74,8 +73,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions)
@GuardedBy("this")
private var currentOffset: Int = -1
- // Exposed for tests.
- private[spark] var startOffset: TextSocketOffset = _
+ private var startOffset: TextSocketOffset = _
private val recordEndpoint = new ContinuousRecordEndpoint(buckets, this)
@volatile private var endpointRef: RpcEndpointRef = _
@@ -96,16 +94,16 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions)
TextSocketOffset(Serialization.read[List[Int]](json))
}
- override def initialOffset(): Offset = {
- startOffset = TextSocketOffset(List.fill(numPartitions)(0))
- startOffset
+ override def setStartOffset(offset: java.util.Optional[Offset]): Unit = {
+ this.startOffset = offset
+ .orElse(TextSocketOffset(List.fill(numPartitions)(0)))
+ .asInstanceOf[TextSocketOffset]
+ recordEndpoint.setStartOffsets(startOffset.offsets)
}
- override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start)
- }
+ override def getStartOffset: Offset = startOffset
- override def fullSchema(): StructType = {
+ override def readSchema(): StructType = {
if (includeTimestamp) {
TextSocketReader.SCHEMA_TIMESTAMP
} else {
@@ -113,10 +111,8 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions)
}
}
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val startOffset = config.asInstanceOf[SimpleStreamingScanConfig]
- .start.asInstanceOf[TextSocketOffset]
- recordEndpoint.setStartOffsets(startOffset.offsets)
+ override def planInputPartitions(): JList[InputPartition[InternalRow]] = {
+
val endpointName = s"TextSocketContinuousReaderEndpoint-${java.util.UUID.randomUUID()}"
endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint)
@@ -136,13 +132,10 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions)
startOffset.offsets.zipWithIndex.map {
case (offset, i) =>
- TextSocketContinuousInputPartition(endpointName, i, offset, includeTimestamp)
- }.toArray
- }
+ TextSocketContinuousInputPartition(
+ endpointName, i, offset, includeTimestamp): InputPartition[InternalRow]
+ }.asJava
- override def createContinuousReaderFactory(
- config: ScanConfig): ContinuousPartitionReaderFactory = {
- TextSocketReaderFactory
}
override def commit(end: Offset): Unit = synchronized {
@@ -197,7 +190,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions)
logWarning(s"Stream closed by $host:$port")
return
}
- TextSocketContinuousReadSupport.this.synchronized {
+ TextSocketContinuousReader.this.synchronized {
currentOffset += 1
val newData = (line,
Timestamp.valueOf(
@@ -228,30 +221,25 @@ case class TextSocketContinuousInputPartition(
driverEndpointName: String,
partitionId: Int,
startOffset: Int,
- includeTimestamp: Boolean) extends InputPartition
-
-
-object TextSocketReaderFactory extends ContinuousPartitionReaderFactory {
+ includeTimestamp: Boolean)
+extends InputPartition[InternalRow] {
- override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[TextSocketContinuousInputPartition]
- new TextSocketContinuousPartitionReader(
- p.driverEndpointName, p.partitionId, p.startOffset, p.includeTimestamp)
- }
+ override def createPartitionReader(): InputPartitionReader[InternalRow] =
+ new TextSocketContinuousInputPartitionReader(driverEndpointName, partitionId, startOffset,
+ includeTimestamp)
}
-
/**
* Continuous text socket input partition reader.
*
* Polls the driver endpoint for new records.
*/
-class TextSocketContinuousPartitionReader(
+class TextSocketContinuousInputPartitionReader(
driverEndpointName: String,
partitionId: Int,
startOffset: Int,
includeTimestamp: Boolean)
- extends ContinuousPartitionReader[InternalRow] {
+ extends ContinuousInputPartitionReader[InternalRow] {
private val endpoint = RpcUtils.makeDriverRef(
driverEndpointName,
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
index a08411d..967dbe2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala
@@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.streaming.continuous
import org.apache.spark.{Partition, SparkEnv, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.sources.v2.writer.DataWriter
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory
+import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory}
import org.apache.spark.util.Utils
/**
@@ -32,7 +31,7 @@ import org.apache.spark.util.Utils
*
* We keep repeating prev.compute() and writing new epochs until the query is shut down.
*/
-class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDataWriterFactory)
+class ContinuousWriteRDD(var prev: RDD[InternalRow], writeTask: DataWriterFactory[InternalRow])
extends RDD[Unit](prev) {
override val partitioner = prev.partitioner
@@ -51,7 +50,7 @@ class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDat
Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
try {
val dataIterator = prev.compute(split, context)
- dataWriter = writerFactory.createWriter(
+ dataWriter = writeTask.createDataWriter(
context.partitionId(),
context.taskAttemptId(),
EpochTracker.getCurrentEpoch.get)
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
index 2238ce2..8877ebe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala
@@ -23,9 +23,9 @@ import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset}
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset}
import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
import org.apache.spark.util.RpcUtils
private[continuous] sealed trait EpochCoordinatorMessage extends Serializable
@@ -82,15 +82,15 @@ private[sql] object EpochCoordinatorRef extends Logging {
* Create a reference to a new [[EpochCoordinator]].
*/
def create(
- writeSupport: StreamingWriteSupport,
- readSupport: ContinuousReadSupport,
+ writer: StreamWriter,
+ reader: ContinuousReader,
query: ContinuousExecution,
epochCoordinatorId: String,
startEpoch: Long,
session: SparkSession,
env: SparkEnv): RpcEndpointRef = synchronized {
val coordinator = new EpochCoordinator(
- writeSupport, readSupport, query, startEpoch, session, env.rpcEnv)
+ writer, reader, query, startEpoch, session, env.rpcEnv)
val ref = env.rpcEnv.setupEndpoint(endpointName(epochCoordinatorId), coordinator)
logInfo("Registered EpochCoordinator endpoint")
ref
@@ -115,8 +115,8 @@ private[sql] object EpochCoordinatorRef extends Logging {
* have both committed and reported an end offset for a given epoch.
*/
private[continuous] class EpochCoordinator(
- writeSupport: StreamingWriteSupport,
- readSupport: ContinuousReadSupport,
+ writer: StreamWriter,
+ reader: ContinuousReader,
query: ContinuousExecution,
startEpoch: Long,
session: SparkSession,
@@ -198,7 +198,7 @@ private[continuous] class EpochCoordinator(
s"and is ready to be committed. Committing epoch $epoch.")
// Sequencing is important here. We must commit to the writer before recording the commit
// in the query, or we will end up dropping the commit if we restart in the middle.
- writeSupport.commit(epoch, messages.toArray)
+ writer.commit(epoch, messages.toArray)
query.commit(epoch)
}
@@ -220,7 +220,7 @@ private[continuous] class EpochCoordinator(
partitionOffsets.collect { case ((e, _), o) if e == epoch => o }
if (thisEpochOffsets.size == numReaderPartitions) {
logDebug(s"Epoch $epoch has offsets reported from all partitions: $thisEpochOffsets")
- query.addOffset(epoch, readSupport, thisEpochOffsets.toSeq)
+ query.addOffset(epoch, reader, thisEpochOffsets.toSeq)
resolveCommitsAtEpoch(epoch)
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala
index 7ad21cc..943c731 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala
@@ -19,13 +19,13 @@ package org.apache.spark.sql.execution.streaming.continuous
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
/**
* The logical plan for writing data in a continuous stream.
*/
case class WriteToContinuousDataSource(
- writeSupport: StreamingWriteSupport, query: LogicalPlan) extends LogicalPlan {
+ writer: StreamWriter, query: LogicalPlan) extends LogicalPlan {
override def children: Seq[LogicalPlan] = Seq(query)
override def output: Seq[Attribute] = Nil
}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
index c216b61..927d3a8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala
@@ -26,21 +26,21 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.streaming.StreamExecution
-import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
/**
- * The physical plan for writing data into a continuous processing [[StreamingWriteSupport]].
+ * The physical plan for writing data into a continuous processing [[StreamWriter]].
*/
-case class WriteToContinuousDataSourceExec(writeSupport: StreamingWriteSupport, query: SparkPlan)
+case class WriteToContinuousDataSourceExec(writer: StreamWriter, query: SparkPlan)
extends SparkPlan with Logging {
override def children: Seq[SparkPlan] = Seq(query)
override def output: Seq[Attribute] = Nil
override protected def doExecute(): RDD[InternalRow] = {
- val writerFactory = writeSupport.createStreamingWriterFactory()
+ val writerFactory = writer.createWriterFactory()
val rdd = new ContinuousWriteRDD(query.execute(), writerFactory)
- logInfo(s"Start processing data source write support: $writeSupport. " +
+ logInfo(s"Start processing data source writer: $writer. " +
s"The input RDD has ${rdd.partitions.length} partitions.")
EpochCoordinatorRef.get(
sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY),
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
index adf52ab..f81abdc 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
@@ -17,9 +17,12 @@
package org.apache.spark.sql.execution.streaming
+import java.{util => ju}
+import java.util.Optional
import java.util.concurrent.atomic.AtomicInteger
import javax.annotation.concurrent.GuardedBy
+import scala.collection.JavaConverters._
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.util.control.NonFatal
@@ -31,8 +34,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
-import org.apache.spark.sql.sources.v2.reader._
-import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2}
+import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
+import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset => OffsetV2}
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
@@ -64,7 +67,7 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas
addData(data.toTraversable)
}
- def fullSchema(): StructType = encoder.schema
+ def readSchema(): StructType = encoder.schema
protected def logicalPlan: LogicalPlan
@@ -77,7 +80,7 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas
* available.
*/
case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
- extends MemoryStreamBase[A](sqlContext) with MicroBatchReadSupport with Logging {
+ extends MemoryStreamBase[A](sqlContext) with MicroBatchReader with Logging {
protected val logicalPlan: LogicalPlan =
StreamingExecutionRelation(this, attributes)(sqlContext.sparkSession)
@@ -119,22 +122,24 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]"
- override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong)
+ override def setOffsetRange(start: Optional[OffsetV2], end: Optional[OffsetV2]): Unit = {
+ synchronized {
+ startOffset = start.orElse(LongOffset(-1)).asInstanceOf[LongOffset]
+ endOffset = end.orElse(currentOffset).asInstanceOf[LongOffset]
+ }
+ }
- override def initialOffset: OffsetV2 = LongOffset(-1)
+ override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong)
- override def latestOffset(): OffsetV2 = {
- if (currentOffset.offset == -1) null else currentOffset
+ override def getStartOffset: OffsetV2 = synchronized {
+ if (startOffset.offset == -1) null else startOffset
}
- override def newScanConfigBuilder(start: OffsetV2, end: OffsetV2): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
+ override def getEndOffset: OffsetV2 = synchronized {
+ if (endOffset.offset == -1) null else endOffset
}
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val sc = config.asInstanceOf[SimpleStreamingScanConfig]
- val startOffset = sc.start.asInstanceOf[LongOffset]
- val endOffset = sc.end.get.asInstanceOf[LongOffset]
+ override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
synchronized {
// Compute the internal batch numbers to fetch: [startOrdinal, endOrdinal)
val startOrdinal = startOffset.offset.toInt + 1
@@ -151,15 +156,11 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
logDebug(generateDebugString(newBlocks.flatten, startOrdinal, endOrdinal))
newBlocks.map { block =>
- new MemoryStreamInputPartition(block)
- }.toArray
+ new MemoryStreamInputPartition(block): InputPartition[InternalRow]
+ }.asJava
}
}
- override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
- MemoryStreamReaderFactory
- }
-
private def generateDebugString(
rows: Seq[UnsafeRow],
startOrdinal: Int,
@@ -200,12 +201,10 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
}
-class MemoryStreamInputPartition(val records: Array[UnsafeRow]) extends InputPartition
-
-object MemoryStreamReaderFactory extends PartitionReaderFactory {
- override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
- val records = partition.asInstanceOf[MemoryStreamInputPartition].records
- new PartitionReader[InternalRow] {
+class MemoryStreamInputPartition(records: Array[UnsafeRow])
+ extends InputPartition[InternalRow] {
+ override def createPartitionReader(): InputPartitionReader[InternalRow] = {
+ new InputPartitionReader[InternalRow] {
private var currentIndex = -1
override def next(): Boolean = {
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala
deleted file mode 100644
index 833e62f..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{Dataset, SparkSession}
-import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
-import org.apache.spark.sql.sources.v2.DataSourceOptions
-import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage
-import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport}
-import org.apache.spark.sql.types.StructType
-
-/** Common methods used to create writes for the the console sink */
-class ConsoleWriteSupport(schema: StructType, options: DataSourceOptions)
- extends StreamingWriteSupport with Logging {
-
- // Number of rows to display, by default 20 rows
- protected val numRowsToShow = options.getInt("numRows", 20)
-
- // Truncate the displayed data if it is too long, by default it is true
- protected val isTruncated = options.getBoolean("truncate", true)
-
- assert(SparkSession.getActiveSession.isDefined)
- protected val spark = SparkSession.getActiveSession.get
-
- def createStreamingWriterFactory(): StreamingDataWriterFactory = PackedRowWriterFactory
-
- override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
- // We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2
- // behavior.
- printRows(messages, schema, s"Batch: $epochId")
- }
-
- def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
-
- protected def printRows(
- commitMessages: Array[WriterCommitMessage],
- schema: StructType,
- printMessage: String): Unit = {
- val rows = commitMessages.collect {
- case PackedRowCommitMessage(rs) => rs
- }.flatten
-
- // scalastyle:off println
- println("-------------------------------------------")
- println(printMessage)
- println("-------------------------------------------")
- // scalastyle:off println
- Dataset.ofRows(spark, LocalRelation(schema.toAttributes, rows))
- .show(numRowsToShow, isTruncated)
- }
-
- override def toString(): String = {
- s"ConsoleWriter[numRows=$numRowsToShow, truncate=$isTruncated]"
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
new file mode 100644
index 0000000..fd45ba5
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.sources
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.sources.v2.DataSourceOptions
+import org.apache.spark.sql.sources.v2.writer.{DataWriterFactory, WriterCommitMessage}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
+import org.apache.spark.sql.types.StructType
+
+/** Common methods used to create writes for the the console sink */
+class ConsoleWriter(schema: StructType, options: DataSourceOptions)
+ extends StreamWriter with Logging {
+
+ // Number of rows to display, by default 20 rows
+ protected val numRowsToShow = options.getInt("numRows", 20)
+
+ // Truncate the displayed data if it is too long, by default it is true
+ protected val isTruncated = options.getBoolean("truncate", true)
+
+ assert(SparkSession.getActiveSession.isDefined)
+ protected val spark = SparkSession.getActiveSession.get
+
+ def createWriterFactory(): DataWriterFactory[InternalRow] = PackedRowWriterFactory
+
+ override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
+ // We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2
+ // behavior.
+ printRows(messages, schema, s"Batch: $epochId")
+ }
+
+ def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+
+ protected def printRows(
+ commitMessages: Array[WriterCommitMessage],
+ schema: StructType,
+ printMessage: String): Unit = {
+ val rows = commitMessages.collect {
+ case PackedRowCommitMessage(rs) => rs
+ }.flatten
+
+ // scalastyle:off println
+ println("-------------------------------------------")
+ println(printMessage)
+ println("-------------------------------------------")
+ // scalastyle:off println
+ Dataset.ofRows(spark, LocalRelation(schema.toAttributes, rows))
+ .show(numRowsToShow, isTruncated)
+ }
+
+ override def toString(): String = {
+ s"ConsoleWriter[numRows=$numRowsToShow, truncate=$isTruncated]"
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
index dbcc448..4a32217 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala
@@ -17,22 +17,26 @@
package org.apache.spark.sql.execution.streaming.sources
+import java.{util => ju}
+import java.util.Optional
import java.util.concurrent.atomic.AtomicInteger
import javax.annotation.concurrent.GuardedBy
+import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import org.json4s.NoTypeHints
import org.json4s.jackson.Serialization
import org.apache.spark.{SparkEnv, TaskContext}
-import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.sql.{Encoder, SQLContext}
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.streaming.{Offset => _, _}
-import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions}
-import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig, ScanConfigBuilder}
-import org.apache.spark.sql.sources.v2.reader.streaming._
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions}
+import org.apache.spark.sql.sources.v2.reader.InputPartition
+import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset}
+import org.apache.spark.sql.types.StructType
import org.apache.spark.util.RpcUtils
/**
@@ -44,9 +48,7 @@ import org.apache.spark.util.RpcUtils
* the specified offset within the list, or null if that offset doesn't yet have a record.
*/
class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPartitions: Int = 2)
- extends MemoryStreamBase[A](sqlContext)
- with ContinuousReadSupportProvider with ContinuousReadSupport {
-
+ extends MemoryStreamBase[A](sqlContext) with ContinuousReader with ContinuousReadSupport {
private implicit val formats = Serialization.formats(NoTypeHints)
protected val logicalPlan =
@@ -57,6 +59,9 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa
@GuardedBy("this")
private val records = Seq.fill(numPartitions)(new ListBuffer[A])
+ @GuardedBy("this")
+ private var startOffset: ContinuousMemoryStreamOffset = _
+
private val recordEndpoint = new ContinuousRecordEndpoint(records, this)
@volatile private var endpointRef: RpcEndpointRef = _
@@ -70,8 +75,15 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa
ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, records(i).size)).toMap)
}
- override def initialOffset(): Offset = {
- ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, 0)).toMap)
+ override def setStartOffset(start: Optional[Offset]): Unit = synchronized {
+ // Inferred initial offset is position 0 in each partition.
+ startOffset = start.orElse {
+ ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, 0)).toMap)
+ }.asInstanceOf[ContinuousMemoryStreamOffset]
+ }
+
+ override def getStartOffset: Offset = synchronized {
+ startOffset
}
override def deserializeOffset(json: String): ContinuousMemoryStreamOffset = {
@@ -86,40 +98,34 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa
)
}
- override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
- new SimpleStreamingScanConfigBuilder(fullSchema(), start)
- }
-
- override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
- val startOffset = config.asInstanceOf[SimpleStreamingScanConfig]
- .start.asInstanceOf[ContinuousMemoryStreamOffset]
+ override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
synchronized {
val endpointName = s"ContinuousMemoryStreamRecordEndpoint-${java.util.UUID.randomUUID()}-$id"
endpointRef =
recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint)
startOffset.partitionNums.map {
- case (part, index) => ContinuousMemoryStreamInputPartition(endpointName, part, index)
- }.toArray
+ case (part, index) =>
+ new ContinuousMemoryStreamInputPartition(
+ endpointName, part, index): InputPartition[InternalRow]
+ }.toList.asJava
}
}
- override def createContinuousReaderFactory(
- config: ScanConfig): ContinuousPartitionReaderFactory = {
- ContinuousMemoryStreamReaderFactory
- }
-
override def stop(): Unit = {
if (endpointRef != null) recordEndpoint.rpcEnv.stop(endpointRef)
}
override def commit(end: Offset): Unit = {}
- // ContinuousReadSupportProvider implementation
+ // ContinuousReadSupport implementation
// This is necessary because of how StreamTest finds the source for AddDataMemory steps.
- override def createContinuousReadSupport(
+ def createContinuousReader(
+ schema: Optional[StructType],
checkpointLocation: String,
- options: DataSourceOptions): ContinuousReadSupport = this
+ options: DataSourceOptions): ContinuousReader = {
+ this
+ }
}
object ContinuousMemoryStream {
@@ -135,16 +141,12 @@ object ContinuousMemoryStream {
/**
* An input partition for continuous memory stream.
*/
-case class ContinuousMemoryStreamInputPartition(
+class ContinuousMemoryStreamInputPartition(
driverEndpointName: String,
partition: Int,
- startOffset: Int) extends InputPartition
-
-object ContinuousMemoryStreamReaderFactory extends ContinuousPartitionReaderFactory {
- override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = {
- val p = partition.asInstanceOf[ContinuousMemoryStreamInputPartition]
- new ContinuousMemoryStreamPartitionReader(p.driverEndpointName, p.partition, p.startOffset)
- }
+ startOffset: Int) extends InputPartition[InternalRow] {
+ override def createPartitionReader: ContinuousMemoryStreamInputPartitionReader =
+ new ContinuousMemoryStreamInputPartitionReader(driverEndpointName, partition, startOffset)
}
/**
@@ -152,10 +154,10 @@ object ContinuousMemoryStreamReaderFactory extends ContinuousPartitionReaderFact
*
* Polls the driver endpoint for new records.
*/
-class ContinuousMemoryStreamPartitionReader(
+class ContinuousMemoryStreamInputPartitionReader(
driverEndpointName: String,
partition: Int,
- startOffset: Int) extends ContinuousPartitionReader[InternalRow] {
+ startOffset: Int) extends ContinuousInputPartitionReader[InternalRow] {
private val endpoint = RpcUtils.makeDriverRef(
driverEndpointName,
SparkEnv.get.conf,
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala
deleted file mode 100644
index 4218fd5..0000000
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.streaming.sources
-
-import org.apache.spark.sql.{ForeachWriter, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
-import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.python.PythonForeachWriter
-import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamingWriteSupportProvider}
-import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage}
-import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport}
-import org.apache.spark.sql.streaming.OutputMode
-import org.apache.spark.sql.types.StructType
-
-/**
- * A [[org.apache.spark.sql.sources.v2.DataSourceV2]] for forwarding data into the specified
- * [[ForeachWriter]].
- *
- * @param writer The [[ForeachWriter]] to process all data.
- * @param converter An object to convert internal rows to target type T. Either it can be
- * a [[ExpressionEncoder]] or a direct converter function.
- * @tparam T The expected type of the sink.
- */
-case class ForeachWriteSupportProvider[T](
- writer: ForeachWriter[T],
- converter: Either[ExpressionEncoder[T], InternalRow => T])
- extends StreamingWriteSupportProvider {
-
- override def createStreamingWriteSupport(
- queryId: String,
- schema: StructType,
- mode: OutputMode,
- options: DataSourceOptions): StreamingWriteSupport = {
- new StreamingWriteSupport {
- override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
- override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
-
- override def createStreamingWriterFactory(): StreamingDataWriterFactory = {
- val rowConverter: InternalRow => T = converter match {
- case Left(enc) =>
- val boundEnc = enc.resolveAndBind(
- schema.toAttributes,
- SparkSession.getActiveSession.get.sessionState.analyzer)
- boundEnc.fromRow
- case Right(func) =>
- func
- }
- ForeachWriterFactory(writer, rowConverter)
- }
-
- override def toString: String = "ForeachSink"
- }
- }
-}
-
-object ForeachWriteSupportProvider {
- def apply[T](
- writer: ForeachWriter[T],
- encoder: ExpressionEncoder[T]): ForeachWriteSupportProvider[_] = {
- writer match {
- case pythonWriter: PythonForeachWriter =>
- new ForeachWriteSupportProvider[UnsafeRow](
- pythonWriter, Right((x: InternalRow) => x.asInstanceOf[UnsafeRow]))
- case _ =>
- new ForeachWriteSupportProvider[T](writer, Left(encoder))
- }
- }
-}
-
-case class ForeachWriterFactory[T](
- writer: ForeachWriter[T],
- rowConverter: InternalRow => T)
- extends StreamingDataWriterFactory {
- override def createWriter(
- partitionId: Int,
- taskId: Long,
- epochId: Long): ForeachDataWriter[T] = {
- new ForeachDataWriter(writer, rowConverter, partitionId, epochId)
- }
-}
-
-/**
- * A [[DataWriter]] which writes data in this partition to a [[ForeachWriter]].
- *
- * @param writer The [[ForeachWriter]] to process all data.
- * @param rowConverter A function which can convert [[InternalRow]] to the required type [[T]]
- * @param partitionId
- * @param epochId
- * @tparam T The type expected by the writer.
- */
-class ForeachDataWriter[T](
- writer: ForeachWriter[T],
- rowConverter: InternalRow => T,
- partitionId: Int,
- epochId: Long)
- extends DataWriter[InternalRow] {
-
- // If open returns false, we should skip writing rows.
- private val opened = writer.open(partitionId, epochId)
-
- override def write(record: InternalRow): Unit = {
- if (!opened) return
-
- try {
- writer.process(rowConverter(record))
- } catch {
- case t: Throwable =>
- writer.close(t)
- throw t
- }
- }
-
- override def commit(): WriterCommitMessage = {
- writer.close(null)
- ForeachWriterCommitMessage
- }
-
- override def abort(): Unit = {}
-}
-
-/**
- * An empty [[WriterCommitMessage]]. [[ForeachWriter]] implementations have no global coordination.
- */
-case object ForeachWriterCommitMessage extends WriterCommitMessage
http://git-wip-us.apache.org/repos/asf/spark/blob/15d2e9d7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala
new file mode 100644
index 0000000..e8ce21c
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala
@@ -0,0 +1,139 @@
+/*
+ * 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.sources
+
+import org.apache.spark.sql.{ForeachWriter, SparkSession}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.execution.python.PythonForeachWriter
+import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamWriteSupport}
+import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory, WriterCommitMessage}
+import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
+import org.apache.spark.sql.streaming.OutputMode
+import org.apache.spark.sql.types.StructType
+
+/**
+ * A [[org.apache.spark.sql.sources.v2.DataSourceV2]] for forwarding data into the specified
+ * [[ForeachWriter]].
+ *
+ * @param writer The [[ForeachWriter]] to process all data.
+ * @param converter An object to convert internal rows to target type T. Either it can be
+ * a [[ExpressionEncoder]] or a direct converter function.
+ * @tparam T The expected type of the sink.
+ */
+case class ForeachWriterProvider[T](
+ writer: ForeachWriter[T],
+ converter: Either[ExpressionEncoder[T], InternalRow => T]) extends StreamWriteSupport {
+
+ override def createStreamWriter(
+ queryId: String,
+ schema: StructType,
+ mode: OutputMode,
+ options: DataSourceOptions): StreamWriter = {
+ new StreamWriter {
+ override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+ override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
+
+ override def createWriterFactory(): DataWriterFactory[InternalRow] = {
+ val rowConverter: InternalRow => T = converter match {
+ case Left(enc) =>
+ val boundEnc = enc.resolveAndBind(
+ schema.toAttributes,
+ SparkSession.getActiveSession.get.sessionState.analyzer)
+ boundEnc.fromRow
+ case Right(func) =>
+ func
+ }
+ ForeachWriterFactory(writer, rowConverter)
+ }
+
+ override def toString: String = "ForeachSink"
+ }
+ }
+}
+
+object ForeachWriterProvider {
+ def apply[T](
+ writer: ForeachWriter[T],
+ encoder: ExpressionEncoder[T]): ForeachWriterProvider[_] = {
+ writer match {
+ case pythonWriter: PythonForeachWriter =>
+ new ForeachWriterProvider[UnsafeRow](
+ pythonWriter, Right((x: InternalRow) => x.asInstanceOf[UnsafeRow]))
+ case _ =>
+ new ForeachWriterProvider[T](writer, Left(encoder))
+ }
+ }
+}
+
+case class ForeachWriterFactory[T](
+ writer: ForeachWriter[T],
+ rowConverter: InternalRow => T)
+ extends DataWriterFactory[InternalRow] {
+ override def createDataWriter(
+ partitionId: Int,
+ taskId: Long,
+ epochId: Long): ForeachDataWriter[T] = {
+ new ForeachDataWriter(writer, rowConverter, partitionId, epochId)
+ }
+}
+
+/**
+ * A [[DataWriter]] which writes data in this partition to a [[ForeachWriter]].
+ *
+ * @param writer The [[ForeachWriter]] to process all data.
+ * @param rowConverter A function which can convert [[InternalRow]] to the required type [[T]]
+ * @param partitionId
+ * @param epochId
+ * @tparam T The type expected by the writer.
+ */
+class ForeachDataWriter[T](
+ writer: ForeachWriter[T],
+ rowConverter: InternalRow => T,
+ partitionId: Int,
+ epochId: Long)
+ extends DataWriter[InternalRow] {
+
+ // If open returns false, we should skip writing rows.
+ private val opened = writer.open(partitionId, epochId)
+
+ override def write(record: InternalRow): Unit = {
+ if (!opened) return
+
+ try {
+ writer.process(rowConverter(record))
+ } catch {
+ case t: Throwable =>
+ writer.close(t)
+ throw t
+ }
+ }
+
+ override def commit(): WriterCommitMessage = {
+ writer.close(null)
+ ForeachWriterCommitMessage
+ }
+
+ override def abort(): Unit = {}
+}
+
+/**
+ * An empty [[WriterCommitMessage]]. [[ForeachWriter]] implementations have no global coordination.
+ */
+case object ForeachWriterCommitMessage extends WriterCommitMessage
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org