Skip to content

Commit a75f927

Browse files
committed
[SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2
## What changes were proposed in this pull request? This is a followup of #20435. While reorganizing the packages for streaming data source v2, the top level stream read/write support interfaces should not be in the reader/writer package, but should be in the `sources.v2` package, to follow the `ReadSupport`, `WriteSupport`, etc. ## How was this patch tested? N/A Author: Wenchen Fan <[email protected]> Closes #20509 from cloud-fan/followup.
1 parent 7f5f5fb commit a75f927

File tree

17 files changed

+19
-33
lines changed

17 files changed

+19
-33
lines changed

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,9 +30,7 @@ import org.apache.spark.internal.Logging
3030
import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext}
3131
import org.apache.spark.sql.execution.streaming.{Sink, Source}
3232
import org.apache.spark.sql.sources._
33-
import org.apache.spark.sql.sources.v2.DataSourceOptions
34-
import org.apache.spark.sql.sources.v2.reader.ContinuousReadSupport
35-
import org.apache.spark.sql.sources.v2.writer.StreamWriteSupport
33+
import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, StreamWriteSupport}
3634
import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
3735
import org.apache.spark.sql.streaming.OutputMode
3836
import org.apache.spark.sql.types.StructType
Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,13 +15,11 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql.sources.v2.reader;
18+
package org.apache.spark.sql.sources.v2;
1919

2020
import java.util.Optional;
2121

2222
import org.apache.spark.annotation.InterfaceStability;
23-
import org.apache.spark.sql.sources.v2.DataSourceV2;
24-
import org.apache.spark.sql.sources.v2.DataSourceOptions;
2523
import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader;
2624
import org.apache.spark.sql.types.StructType;
2725

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,13 +15,11 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql.sources.v2.reader;
18+
package org.apache.spark.sql.sources.v2;
1919

2020
import java.util.Optional;
2121

2222
import org.apache.spark.annotation.InterfaceStability;
23-
import org.apache.spark.sql.sources.v2.DataSourceOptions;
24-
import org.apache.spark.sql.sources.v2.DataSourceV2;
2523
import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
2624
import org.apache.spark.sql.types.StructType;
2725

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,12 +15,11 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql.sources.v2.writer;
18+
package org.apache.spark.sql.sources.v2;
1919

2020
import org.apache.spark.annotation.InterfaceStability;
2121
import org.apache.spark.sql.execution.streaming.BaseStreamingSink;
22-
import org.apache.spark.sql.sources.v2.DataSourceOptions;
23-
import org.apache.spark.sql.sources.v2.DataSourceV2;
22+
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
2423
import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter;
2524
import org.apache.spark.sql.streaming.OutputMode;
2625
import org.apache.spark.sql.types.StructType;

sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import org.apache.spark.sql.Row;
2222
import org.apache.spark.sql.SaveMode;
2323
import org.apache.spark.sql.sources.v2.DataSourceOptions;
24+
import org.apache.spark.sql.sources.v2.StreamWriteSupport;
2425
import org.apache.spark.sql.sources.v2.WriteSupport;
2526
import org.apache.spark.sql.streaming.OutputMode;
2627
import org.apache.spark.sql.types.StructType;

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
2929
import org.apache.spark.sql.execution.SQLExecution
3030
import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2}
3131
import org.apache.spark.sql.execution.streaming.sources.{InternalRowMicroBatchWriter, MicroBatchWriter}
32-
import org.apache.spark.sql.sources.v2.DataSourceOptions
33-
import org.apache.spark.sql.sources.v2.reader.MicroBatchReadSupport
32+
import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport}
3433
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset => OffsetV2}
35-
import org.apache.spark.sql.sources.v2.writer.{StreamWriteSupport, SupportsWriteInternalRow}
34+
import org.apache.spark.sql.sources.v2.writer.SupportsWriteInternalRow
3635
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
3736
import org.apache.spark.util.{Clock, Utils}
3837

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
3232
import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReader
3333
import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
3434
import org.apache.spark.sql.sources.v2._
35-
import org.apache.spark.sql.sources.v2.reader.ContinuousReadSupport
3635
import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader
3736
import org.apache.spark.sql.types._
3837
import org.apache.spark.util.{ManualClock, SystemClock}

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LeafNode
2525
import org.apache.spark.sql.catalyst.plans.logical.Statistics
2626
import org.apache.spark.sql.execution.LeafExecNode
2727
import org.apache.spark.sql.execution.datasources.DataSource
28-
import org.apache.spark.sql.sources.v2.DataSourceV2
29-
import org.apache.spark.sql.sources.v2.reader.ContinuousReadSupport
28+
import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2}
3029

3130
object StreamingRelation {
3231
def apply(dataSource: DataSource): StreamingRelation = {

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.streaming
2020
import org.apache.spark.sql._
2121
import org.apache.spark.sql.execution.streaming.sources.ConsoleWriter
2222
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister}
23-
import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2}
24-
import org.apache.spark.sql.sources.v2.writer.StreamWriteSupport
23+
import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport}
2524
import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
2625
import org.apache.spark.sql.streaming.OutputMode
2726
import org.apache.spark.sql.types.StructType

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,10 +31,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
3131
import org.apache.spark.sql.execution.SQLExecution
3232
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, StreamingDataSourceV2Relation, WriteToDataSourceV2}
3333
import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _}
34-
import org.apache.spark.sql.sources.v2.DataSourceOptions
35-
import org.apache.spark.sql.sources.v2.reader.ContinuousReadSupport
34+
import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, StreamWriteSupport}
3635
import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset}
37-
import org.apache.spark.sql.sources.v2.writer.StreamWriteSupport
3836
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
3937
import org.apache.spark.sql.types.StructType
4038
import org.apache.spark.util.{Clock, Utils}

0 commit comments

Comments
 (0)