diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala index 3171f1e08b4f..c6f52d676422 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.avro.AvroFileFormat +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index 243af7da4700..0397d15aed92 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -31,10 +31,10 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index 6ec351080a11..e1268ac2ce58 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -21,9 +21,9 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala index 815da2bd92d4..e36c71ef4b1f 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala index a781624aa61a..765e5727d944 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala @@ -22,9 +22,9 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroUtils +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index 700414167f3e..b958035b3922 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -23,8 +23,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory} - +import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory} private[kafka010] class KafkaBatch( strategy: ConsumerStrategy, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala index 53b0b3c46854..d4aa6774be66 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -22,8 +22,7 @@ import java.{util => ju} 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.sources.v2.reader._ - +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} /** A [[InputPartition]] for reading Kafka data in a batch based streaming query. */ private[kafka010] case class KafkaBatchInputPartition( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala index 47ec07ae128d..8e29e38b2a64 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery -import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.types.StructType /** diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index a9c1181a01c5..1e8da4bc0f7c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -27,9 +27,9 @@ 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.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} 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.util.CaseInsensitiveStringMap /** diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala index 884773452b2a..3f8d3d2da579 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala @@ -21,7 +21,7 @@ import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} /** * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 9cd16c8e1624..52d91abc8621 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -26,10 +26,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream 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.{MicroBatchStream, Offset} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.UninterruptibleThread diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 90d70439c532..b9674a30aee3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.connector.read.streaming.PartitionOffset import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and 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 c3f0be4be96e..53a6919e2d4b 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,14 +30,13 @@ import org.apache.spark.internal.Logging import org.apache.spark.kafka010.KafkaConfigUpdater import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Batch, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} +import org.apache.spark.sql.connector.write.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite 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.TableCapability._ -import org.apache.spark.sql.sources.v2.reader.{Batch, Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -367,6 +366,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister override def schema(): StructType = KafkaOffsetReader.kafkaSchema override def capabilities(): ju.Set[TableCapability] = { + import TableCapability._ // ACCEPT_ANY_SCHEMA is needed because of the following reasons: // * Kafka writer validates the schema instead of the SQL analyzer (the schema is fixed) // * Read schema differs from write schema (please see Kafka integration guide) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala index 6dd1d2984a96..2b50b771e694 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} 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, StreamingWrite} import org.apache.spark.sql.types.StructType /** 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 ae8a6886b2b4..9fe100431c01 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 @@ -35,6 +35,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ @@ -42,7 +43,6 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.kafka010.KafkaSourceProvider._ -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSparkSession diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala index 8e6de88865e0..f7b00b31ebba 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.mockito.Mockito.{mock, when} import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} -import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.util.CaseInsensitiveStringMap class KafkaSourceProviderSuite extends SparkFunSuite { diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index d254508e5d35..d0eeaa5c6218 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -1110,7 +1110,7 @@ def test_datasource_with_udf(self): .format("org.apache.spark.sql.sources.SimpleScanSource") \ .option('from', 0).option('to', 1).load().toDF('i') datasource_v2_df = self.spark.read \ - .format("org.apache.spark.sql.sources.v2.SimpleDataSourceV2") \ + .format("org.apache.spark.sql.connector.SimpleDataSourceV2") \ .load().toDF('i', 'j') c1 = pandas_udf(lambda x: x + 1, 'int')(lit(1)) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 4a0a3760a556..c274dc73225b 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -552,7 +552,7 @@ def test_datasource_with_udf(self): .format("org.apache.spark.sql.sources.SimpleScanSource") \ .option('from', 0).option('to', 1).load().toDF('i') datasource_v2_df = self.spark.read \ - .format("org.apache.spark.sql.sources.v2.SimpleDataSourceV2") \ + .format("org.apache.spark.sql.connector.SimpleDataSourceV2") \ .load().toDF('i', 'j') c1 = udf(lambda x: x + 1, 'int')(lit(1)) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java index 2bf72c1da73d..32ba24ff0256 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java index 5d4995a05d23..615aab3ef325 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.internal.SQLConf; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java index f471a4e71cf4..b6a896c52d53 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.SparkException; import org.apache.spark.annotation.Private; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java index 2d3700a08806..d7f0ee15f840 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; import org.apache.spark.annotation.Experimental; -import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java index 3e697c1945bf..c3ac9c20db3b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java index 34f3882c9c41..56d13ef742ce 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import com.google.common.base.Preconditions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java index 6f5895bcc380..1db6877c8d0b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; + +import org.apache.spark.annotation.Experimental; /** * NamespaceChange subclasses represent requested changes to a namespace. These are passed to @@ -29,6 +31,7 @@ * ) * */ +@Experimental public interface NamespaceChange { /** * Create a NamespaceChange for setting a namespace property. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java index d27fbfdd1461..fc076ee00a62 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java similarity index 90% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java index b2baa93b146a..9fd70cc97700 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; -import org.apache.spark.sql.catalog.v2.Identifier; -import org.apache.spark.sql.catalog.v2.StagingTableCatalog; -import org.apache.spark.sql.catalog.v2.expressions.Transform; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -37,6 +37,7 @@ * at which point implementations are expected to commit the table's metadata into the metastore * along with the data that was written by the writes from the write builder this table created. */ +@Experimental public interface StagedTable extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index fc055e91a6ac..ca9160aa2f16 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.StagedTable; -import org.apache.spark.sql.sources.v2.SupportsWrite; -import org.apache.spark.sql.sources.v2.writer.BatchWrite; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriterCommitMessage; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -50,6 +49,7 @@ * {@link StagedTable#commitStagedChanges()} is called, at which point the staged table can * complete both the data write and the metadata swap operation atomically. */ +@Experimental public interface StagingTableCatalog extends TableCatalog { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java index 8650a0ef1d4b..80aa57ca1877 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java @@ -15,14 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.sources.Filter; /** * A mix-in interface for {@link Table} delete support. Data sources can implement this * interface to provide the ability to delete data from tables that matches filter expressions. */ +@Experimental public interface SupportsDelete { /** * Delete data from a data source table that matches filter expressions. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 12c2e511f33f..6845f9294a6d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; @@ -35,6 +36,7 @@ * drop a namespace. Implementations are allowed to discover the existence of objects or namespaces * without throwing {@link NoSuchNamespaceException} when no namespace is found. */ +@Experimental public interface SupportsNamespaces extends CatalogPlugin { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java similarity index 87% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java index 826fa2f8a072..cdff1a38ff05 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.sources.v2.reader.Scan; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** @@ -26,6 +27,7 @@ * {@link #newScanBuilder(CaseInsensitiveStringMap)} that is used to create a scan for batch, * micro-batch, or continuous processing. */ +@Experimental public interface SupportsRead extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java similarity index 85% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java index c52e54569dc0..5b648468c84a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.sources.v2.writer.BatchWrite; -import org.apache.spark.sql.sources.v2.writer.WriteBuilder; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** @@ -26,6 +27,7 @@ * {@link #newWriteBuilder(CaseInsensitiveStringMap)} that is used to create a write * for batch or streaming. */ +@Experimental public interface SupportsWrite extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index 21e1e8c533f3..a493736c7727 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import java.util.Collections; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java similarity index 89% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java index eccf2892b039..9765118a8dbf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; @@ -24,7 +24,7 @@ *

* Tables use {@link Table#capabilities()} to return a set of capabilities. Each capability signals * to Spark that the table supports a feature identified by the capability. For example, returning - * {@code BATCH_READ} allows Spark to read from the table using a batch scan. + * {@link #BATCH_READ} allows Spark to read from the table using a batch scan. */ @Experimental public enum TableCapability { @@ -66,7 +66,7 @@ public enum TableCapability { *

* Truncating a table removes all existing rows. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsTruncate}. + * See {@link org.apache.spark.sql.connector.write.SupportsTruncate}. */ TRUNCATE, @@ -74,7 +74,7 @@ public enum TableCapability { * Signals that the table can replace existing data that matches a filter with appended data in * a write operation. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsOverwrite}. + * See {@link org.apache.spark.sql.connector.write.SupportsOverwrite}. */ OVERWRITE_BY_FILTER, @@ -82,7 +82,7 @@ public enum TableCapability { * Signals that the table can dynamically replace existing data partitions with appended data in * a write operation. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsDynamicOverwrite}. + * See {@link org.apache.spark.sql.connector.write.SupportsDynamicOverwrite}. */ OVERWRITE_DYNAMIC, diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 4775b58edf04..b72f7845a6ea 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.Table; import org.apache.spark.sql.types.StructType; import java.util.Map; @@ -35,6 +35,7 @@ * table schema when updating, renaming, or dropping existing columns when catalyst analysis is case * insensitive. */ +@Experimental public interface TableCatalog extends CatalogPlugin { /** * List the tables in a namespace from the catalog. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 7eef6aea8812..88ab51c1d70b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; /** @@ -32,6 +33,7 @@ * ) * */ +@Experimental public interface TableChange { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index 1d37ff042bd3..e9fd87d0e2d4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java index 1e2aca9556df..807731ac2294 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java index 7b264e7480e1..bf2a1bab7bf4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java index e41bcf9000c5..942744bf6a5d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java index c71ffbe70651..e6e5fa59c249 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java index c85e0c412f1a..edac6df8a4d3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java index 28d80b7a5bc3..09592598a82f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java index 413349782efa..bedf30803f73 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java index 294592595953..254ee56501b9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Closeable; import java.io.IOException; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java index 97f4a473953f..7fef69ce2a5b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java index c3964e2176d4..c1584a58c117 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousStream; -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchStream; +import org.apache.spark.sql.connector.read.streaming.ContinuousStream; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; /** * A logical representation of a data source scan. This interface is used to provide logical diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java index d4bc1ff97713..ca2cd59b2e44 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java index a0b194a41f58..42778b4e28a1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.util.OptionalLong; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java index f10fd884daab..bee9e5508ca6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.Filter; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java index 862bd14bffed..97143686d3ef 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java index 4ce97bc5e76b..5bbc19173039 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; +import org.apache.spark.sql.connector.read.partitioning.Partitioning; /** * A mix in interface for {@link Scan}. Data sources can implement this interface to diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java index d7364af69e89..3cd5ef60dc53 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java index 1cdc02f5736b..23d3c6b563d1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * A concrete implementation of {@link Distribution}. Represents a distribution where records that diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java index 02b0e6897491..fd397c760f05 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * An interface to represent data distribution requirement, which specifies how the records should diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java similarity index 90% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java index c7370eb3d38a..f0b9410ee6e6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.SupportsReportPartitioning; /** * An interface to represent the output data partitioning for a data source, which is returned by diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java similarity index 92% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java index c7f6fce6e81a..8bd5273bb7d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * A variation on {@link PartitionReader} for use with continuous streaming processing. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java similarity index 81% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java index 41195befe5e5..962864da4aad 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; 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.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.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. + * instead of {@link PartitionReader}. It's used for continuous streaming processing. */ @Evolving public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java index fff5b95a4de1..ee01a2553ae7 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.Scan; /** * A {@link SparkDataStream} for streaming queries with continuous mode. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java similarity index 86% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java index 330f07ba4f2f..ceab0f75734d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; -import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; /** * A {@link SparkDataStream} for streaming queries with micro-batch mode. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java index 1d34fdd1c28a..400de2a65974 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java index 2c97d924a062..35ad3bbde5cb 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java index 2068a84fc6bb..1ba0c25ef446 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java index 91297759971b..37c5539d2518 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java index 11228ad1ea67..eefe784dede4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.IOException; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java index bf2db9059b08..bcf8d8a59e5e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java index 8058964b662b..90e668d1935b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; /** * Write builder trait for tables that support dynamic partition overwrite. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java index b443b3c3aeb4..6063a155ee20 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.sql.sources.AlwaysTrue$; import org.apache.spark.sql.sources.Filter; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java index 69c2ba5e01a4..15a68375032d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; /** * Write builder trait for tables that support truncation. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java index 158066d40292..f26304e8db7b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.write.streaming.StreamingWrite; import org.apache.spark.sql.types.StructType; /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java similarity index 92% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java index 23e8580c404d..823f955bffb5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.Serializable; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; +import org.apache.spark.sql.connector.write.streaming.StreamingWrite; /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java index af2f03c9d419..daaa18d5bc4e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer.streaming; +package org.apache.spark.sql.connector.write.streaming; import java.io.Serializable; import org.apache.spark.TaskContext; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataWriter; +import org.apache.spark.sql.connector.write.DataWriter; /** * A factory of {@link DataWriter} returned by diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java index 5617f1cdc0ef..0821b3489165 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer.streaming; +package org.apache.spark.sql.connector.write.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.WriterCommitMessage; /** * An interface that defines how to write the data to data source in streaming queries. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala index f5e9a146bf35..7e5d56a7d119 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ -import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.Identifier /** * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 73b9c368e313..8e6be32bcf70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -25,8 +25,6 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2._ -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -40,11 +38,11 @@ import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PartitionOverwriteMode, StoreAssignmentPolicy} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -925,7 +923,7 @@ class Analyzer( * the table identifier does not include a catalog. */ object ResolveAlterTable extends Rule[LogicalPlan] { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case alter @ AlterTableAddColumnsStatement(tableName, cols) => val changes = cols.map { col => @@ -2855,7 +2853,7 @@ class Analyzer( private def lookupV2Relation( identifier: Seq[String] ): Either[(CatalogPlugin, Identifier, Option[Table]), Option[Table]] = { - import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ identifier match { case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala index 3036f7c21093..123d8aff25fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.connector.catalog.Identifier class CannotReplaceMissingTableException( tableIdentifier: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 8ad31342fc9b..4a19806bd80f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -26,6 +25,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableStatement, InsertIntoStatement} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -34,7 +34,7 @@ import org.apache.spark.sql.types._ */ trait CheckAnalysis extends PredicateHelper { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** * Override to provide additional checks for correct analysis. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 7ac8ae61ed53..9b5b059908c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ -import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.Identifier /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 9e0e0d528a96..40f25fb7f972 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -42,7 +42,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str */ case class UnresolvedRelation( multipartIdentifier: Seq[String]) extends LeafNode with NamedRelation { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** Returns a `.` separated name for this relation. */ def tableName: String = multipartIdentifier.quoted diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 25bfaa89017c..8556ff477553 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -28,8 +28,6 @@ import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2 -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} @@ -40,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -2177,7 +2176,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTransformList(ctx: TransformListContext): Seq[Transform] = withOrigin(ctx) { def getFieldReference( ctx: ApplyTransformContext, - arg: v2.expressions.Expression): FieldReference = { + arg: V2Expression): FieldReference = { lazy val name: String = ctx.identifier.getText arg match { case ref: FieldReference => @@ -2190,7 +2189,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging def getSingleFieldReference( ctx: ApplyTransformContext, - arguments: Seq[v2.expressions.Expression]): FieldReference = { + arguments: Seq[V2Expression]): FieldReference = { lazy val name: String = ctx.identifier.getText if (arguments.size > 1) { throw new ParseException(s"Too many arguments for transform $name", ctx) @@ -2247,7 +2246,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Parse an argument to a transform. An argument may be a field reference (qualified name) or * a value literal. */ - override def visitTransformArgument(ctx: TransformArgumentContext): v2.expressions.Expression = { + override def visitTransformArgument(ctx: TransformArgumentContext): V2Expression = { withOrigin(ctx) { val reference = Option(ctx.qualifiedName) .map(typedVisit[Seq[String]]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 9806273f379c..017f31760b58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -17,18 +17,17 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalog.v2.{Identifier, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala index 190711303e32..551a0ce44b4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.catalyst.plans.logical.sql -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala index 2808892b089b..b9767ff9b889 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.catalyst.plans.logical.sql -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 5bba88dbe76d..3cf8cec84d0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import scala.collection.mutable import scala.util.control.NonFatal @@ -27,6 +27,7 @@ import org.apache.spark.sql.internal.SQLConf * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow * the caller to look up a catalog by name. */ +private[sql] class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends Logging { private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] @@ -114,6 +115,6 @@ class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends } } -object CatalogManager { +private[sql] object CatalogManager { val SESSION_CATALOG_NAME: String = "session" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala similarity index 95% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala index 86de1c9285b7..d376b98afa41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.SparkException import org.apache.spark.annotation.Experimental diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala similarity index 94% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 87070fb2ad37..7d0acceec37d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.types.StructType /** * Conversion helpers for working with v2 [[CatalogPlugin]]. */ -object CatalogV2Implicits { +private[sql] object CatalogV2Implicits { implicit class PartitionTypeHelper(partitionType: StructType) { def asTransforms: Array[Transform] = partitionType.names.map(LogicalExpressions.identity) } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index d5079202c8fe..a61a55b22285 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -15,21 +15,19 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.utils +package org.apache.spark.sql.connector.catalog import java.util import java.util.Collections import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, NamespaceChange, TableChange} -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} -object CatalogV2Util { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ +private[sql] object CatalogV2Util { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** * Apply properties changes to a map and return the result. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 2281402740a3..baea8204d9be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier /** * A trait to encapsulate catalog lookup function and helpful extractors. */ -@Experimental -trait LookupCatalog extends Logging { +private[sql] trait LookupCatalog extends Logging { protected val catalogManager: CatalogManager diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala index ddfc77d0cc0a..616c3cf69639 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala @@ -15,23 +15,22 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.internal +package org.apache.spark.sql.connector.catalog import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.types.StructType /** * An implementation of catalog v2 `Table` to expose v1 table metadata. */ -case class V1Table(v1Table: CatalogTable) extends Table { +private[sql] case class V1Table(v1Table: CatalogTable) extends Table { implicit class IdentifierHelper(identifier: TableIdentifier) { def quoted: String = { identifier.database match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala similarity index 98% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala index bceea147dddd..2dd0f34c6c05 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions +package org.apache.spark.sql.connector.expressions import org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -253,7 +253,7 @@ private[sql] final case class LiteralValue[T](value: T, dataType: DataType) exte } private[sql] final case class FieldReference(parts: Seq[String]) extends NamedReference { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.MultipartIdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper override def fieldNames: Array[String] = parts.toArray override def describe: String = parts.quoted override def toString: String = describe diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 2d59c42ee868..74842e762b40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.sources.v2.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} object DataSourceV2Implicits { implicit class TableHelper(table: Table) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 9ae3dbbc4550..7da502fc298e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Statistics => V2Statistics, _} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.catalog.{Table, TableCapability} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2Statistics, SupportsReportStatistics} +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index d15440632f32..27b5eec27281 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.util import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java similarity index 99% rename from sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java rename to sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java index 6bd919281699..37f60511cd60 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.SparkException; import org.apache.spark.sql.internal.SQLConf; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index c4e7194f7d65..79e6245528de 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.LogicalExpressions import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LeafNode} import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.LogicalExpressions import org.apache.spark.sql.types.{DoubleType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 2e5ff8c3ce6d..2cb01d4e8aa5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 0dea1e3a68dc..04ae27df9453 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -22,12 +22,12 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.expressions.{IdentityTransform, Transform} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.{IdentityTransform, Transform} +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.sources.{And, EqualTo, Filter, IsNotNull} -import org.apache.spark.sql.sources.v2.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index 8b14ad0d372d..8724a38d08d1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -22,16 +22,14 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class BasicInMemoryTableCatalog extends TableCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() @@ -112,7 +110,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { } class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamespaces { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ protected val namespaces: util.Map[List[String], Map[String, String]] = new ConcurrentHashMap[List[String], Map[String, String]]() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala index e1b0e6136de9..513ea67b1fe9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala @@ -19,18 +19,17 @@ package org.apache.spark.sql.connector import java.util -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, TableAlreadyExistsException} -import org.apache.spark.sql.sources.v2.{StagedTable, SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTableCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ import InMemoryTableCatalog._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def stageCreate( ident: Identifier, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala similarity index 96% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index d2a2ba50ead4..58e2f410bf20 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.catalog +package org.apache.spark.sql.connector.catalog import java.util import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalog.v2.{CatalogManager, NamespaceChange, SupportsNamespaces} import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala similarity index 94% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index 229ed22b507a..187551dd90db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.catalyst.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, when} @@ -23,12 +23,11 @@ import org.scalatest.Inside import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.util.CaseInsensitiveStringMap -private case class TestCatalogPlugin(override val name: String) extends CatalogPlugin { +private case class DummyCatalogPlugin(override val name: String) extends CatalogPlugin { override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = Unit } @@ -36,7 +35,7 @@ private case class TestCatalogPlugin(override val name: String) extends CatalogP class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap + private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) @@ -128,7 +127,7 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap + private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala similarity index 99% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala index 3904790a5e5c..265a940d66a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import java.util import java.util.Collections diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala similarity index 98% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala index c0a5dada19db..fbd6a886d011 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions +package org.apache.spark.sql.connector.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java index 7c167dc01232..707812e5af61 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java @@ -18,10 +18,10 @@ package org.apache.spark.sql.execution.streaming; /** - * This class is an alias of {@link org.apache.spark.sql.sources.v2.reader.streaming.Offset}. It's + * This class is an alias of {@link org.apache.spark.sql.connector.read.streaming.Offset}. It's * internal and deprecated. New streaming data source implementations should use data source v2 API, * which will be supported in the long term. * * This class will be removed in a future release. */ -public abstract class Offset extends org.apache.spark.sql.sources.v2.reader.streaming.Offset {} +public abstract class Offset extends org.apache.spark.sql.connector.read.streaming.Offset {} 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 f901005ad4fc..b9cc25817d2f 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 @@ -32,14 +32,14 @@ import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, Univocit import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.FailureSafeParser +import org.apache.spark.sql.connector.catalog.SupportsRead +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.csv._ 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, DataSourceV2Utils} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String 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 4489aece505d..58acfb836b30 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 @@ -22,8 +22,6 @@ import java.util.{Locale, Properties} import scala.collection.JavaConverters._ import org.apache.spark.annotation.Stable -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ @@ -31,15 +29,15 @@ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, LiteralValue, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -341,7 +339,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def insertInto(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ assertNotBucketed("insertInto") @@ -373,7 +371,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val table = catalog.asTableCatalog.loadTable(ident) match { case _: V1Table => @@ -484,7 +482,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def saveAsTable(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala rename to sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala index 2a88555e2927..e738ad1ede44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer +package org.apache.spark.sql.connector.write import org.apache.spark.annotation.{Experimental, Unstable} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.sources.InsertableRelation -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * A trait that should be implemented by V1 DataSources that would like to leverage the DataSource diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 9376f0835179..0f5f1591623a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.TableProvider import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat @@ -46,7 +47,6 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{CalendarIntervalType, StructField, StructType} import org.apache.spark.sql.util.SchemaUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 68d32059bfe3..b9b8e069d523 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -20,15 +20,15 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowDatabasesCommand, ShowTablesCommand} +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, LookupCatalog, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -38,7 +38,7 @@ case class DataSourceResolution( catalogManager: CatalogManager) extends Rule[LogicalPlan] with CastSupport with LookupCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTableStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 2595cc6371bc..c1ebc98fb1dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} import org.apache.spark.sql.types.StringType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index 14bee173cc11..b4fc94e097aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.Partition import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.InputPartition +import org.apache.spark.sql.connector.read.InputPartition /** * A collection of file blocks that should be read as a single task diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 9e98b0bbfabc..542c996a5342 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -19,17 +19,14 @@ package org.apache.spark.sql.execution.datasources import java.io.{FileNotFoundException, IOException} -import scala.collection.mutable - import org.apache.parquet.io.ParquetDecodingException -import org.apache.spark.{Partition => RDDPartition, TaskContext, TaskKilledException} +import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index e4f9e49c4dd2..3f4f29c3e135 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -22,10 +22,10 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 615e8a5f8a69..c92c68095db8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala index a3fa82b12e93..4525f2608b71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.SparkException import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, TableChange} import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index 0f98d9486bbb..e4e7887017a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} /** * Physical plan node for scanning a batch of data from a data source v2. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala index f54ff608a53e..dc95d157e40f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector.read.{InputPartition, Scan} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, Offset} import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, Offset} /** * Physical plan node for scanning data from a streaming data source with continuous mode. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index f35758bf08c6..600ed6f7c98e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType @@ -35,7 +35,7 @@ case class CreateTableExec( partitioning: Seq[Transform], tableProperties: Map[String, String], ignoreIfExists: Boolean) extends LeafExecNode { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override protected def doExecute(): RDD[InternalRow] = { if (!catalog.tableExists(identifier)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala index 33079d591250..9211ec25525f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression} import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Partitioning} +import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Partitioning} /** * An adapter from public data source partitioning to catalyst internal `Partitioning`. 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 f62f7349d1da..71560f60f531 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 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.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) extends Partition with Serializable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala index 74fc5432ea82..211f61279ddd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.AttributeMap import org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils 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 10dcf402ed61..22100c7d3d59 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 @@ -17,24 +17,20 @@ package org.apache.spark.sql.execution.datasources.v2 -import java.util.UUID - import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} -import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables} +import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} 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 -import org.apache.spark.sql.sources.v2.TableCapability -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} -import org.apache.spark.sql.sources.v2.writer.V1WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Strategy extends Strategy with PredicateHelper { 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 30897d86f817..52294ae2cb85 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 @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.regex.Pattern import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, TableProvider} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{SessionConfigSupport, TableProvider} private[sql] object DataSourceV2Utils extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala index a5840571fff2..b91cbca8d2e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.SupportsDelete import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.SupportsDelete import org.apache.spark.sql.util.CaseInsensitiveStringMap case class DeleteFromTableExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 640bdfb8cba5..e696efe950f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -24,8 +24,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.LeafExecNode -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType case class DescribeTableExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala index d325e0205f9d..9164b211ed0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala index b177d15e1fe3..711bd41e1db2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import java.io.IOException -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader /** * A [[PartitionReader]] with empty output. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala index db31927fa73b..e7d9a247533c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -20,10 +20,9 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.util.SerializableConfiguration class FileBatchWrite( job: Job, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index ac786bbaac6d..e0091293d166 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -20,9 +20,9 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.TableProvider import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index 836eae88e4da..1b55c181c7ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -22,10 +22,10 @@ import org.apache.parquet.io.ParquetDecodingException import org.apache.spark.internal.Logging import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader class FilePartitionReader[T](readers: Iterator[PartitionedFileReader[T]]) extends PartitionReader[T] with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala index 5a19412c9033..c1d91736a8b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -17,9 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningUtils} -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} import org.apache.spark.sql.vectorized.ColumnarBatch abstract class FilePartitionReaderFactory extends PartitionReaderFactory { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 0438bd0430da..55104a2b21de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -25,10 +25,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala index 3b236be90e6f..21bc14c577bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownRequiredColumns} import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitioningUtils} -import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownRequiredColumns} import org.apache.spark.sql.types.StructType abstract class FileScanBuilder( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 4483f5b1dd30..5329e09916bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -23,11 +23,11 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex} -import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.util.SchemaUtils @@ -39,7 +39,7 @@ abstract class FileTable( userSpecifiedSchema: Option[StructType]) extends Table with SupportsRead with SupportsWrite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ lazy val fileIndex: PartitioningAwareFileIndex = { val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala index eacc4cb3ac4a..65a2c61ba090 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala @@ -30,10 +30,10 @@ import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.connector.write.{BatchWrite, WriteBuilder} import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, DataSource, OutputWriterFactory, WriteJobDescription} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.util.SchemaUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala index eb573b317142..1f25fed3000b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -23,9 +23,8 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory} import org.apache.spark.sql.execution.datasources.{DynamicPartitionDataWriter, SingleDirectoryDataWriter, WriteJobDescription} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} -import org.apache.spark.util.SerializableConfiguration case class FileWriterFactory ( description: WriteJobDescription, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala index a9b0f5bce1b0..bca28e3cacb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan} -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} /** * Physical plan node for scanning a micro-batch of data from a data source. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala index f9dfcf448a3e..0d9aa5b42a6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader class PartitionReaderFromIterator[InternalRow]( iter: Iterator[InternalRow]) extends PartitionReader[InternalRow] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala index 072465b56857..7bca98e54efa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala index baa8cb6b2465..8e524a986aa0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.hadoop.mapreduce.RecordReader -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader class PartitionRecordReader[T]( private[this] var rowReader: RecordReader[_, T]) extends PartitionReader[T] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 35d86ee2abbb..a56ccd6a6a9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.LeafExecNode -import org.apache.spark.sql.sources.v2.StagedTable import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index f70b943fd2cd..49f55be52de1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper -import org.apache.spark.sql.catalog.v2.SupportsNamespaces import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.execution.LeafExecNode /** @@ -37,6 +37,7 @@ case class ShowNamespacesExec( namespace: Option[Seq[String]], pattern: Option[String]) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = { val namespaces = namespace.map { ns => if (ns.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index c652f28a5e76..e352bc675962 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper -import org.apache.spark.sql.catalog.v2.TableCatalog import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 660b6e763e05..51993b1e9ad6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.BooleanType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala index 2f05ff3a7c2e..bf67e972976b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala @@ -19,18 +19,16 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.UUID -import scala.collection.JavaConverters._ - import org.apache.spark.SparkException import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, SaveMode} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.SupportsWrite +import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.sources.{AlwaysTrue, CreatableRelationProvider, Filter, InsertableRelation} -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.sources.{AlwaysTrue, Filter, InsertableRelation} import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 75320fb51db3..525d9c4e541e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -23,18 +23,14 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.NamespaceChange.{RemoveProperty, SetProperty} -import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogDatabase, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.NamespaceChange.RemoveProperty +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.internal.{SessionState, SQLConf} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.internal.V1Table +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -43,7 +39,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap */ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) extends TableCatalog with SupportsNamespaces { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import V2SessionCatalog._ override val defaultNamespace: Array[String] = Array("default") 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 0131d72ebc97..9f4392da6ab4 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 @@ -26,16 +26,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.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, SupportsWrite, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} -import org.apache.spark.sql.sources.v2.{StagedTable, SupportsWrite} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{LongAccumulator, Utils} @@ -69,7 +68,7 @@ case class CreateTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, ifNotExists: Boolean) extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { if (catalog.tableExists(ident)) { @@ -158,7 +157,7 @@ case class ReplaceTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, orCreate: Boolean) extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { // Note that this operation is potentially unsafe, but these are the strict semantics of @@ -470,7 +469,7 @@ object DataWritingSparkTask extends Logging { } private[v2] trait AtomicTableWriteExec extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper protected def writeToStagedTable( stagedTable: StagedTable, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala index 045f41e670ad..1f99d4282f6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.csv +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 828594ffb10a..a20b0f1560a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 3cbcfca01a9c..5125de9313a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.csv.{CSVDataSource, MultiLineCSVDataSource} +import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 28c5b3d81a3d..8b486d034450 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala index 8170661a7017..04beee0e3b0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{AtomicType, DataType, StructType, UserDefinedType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala index 610bd4c1b9d8..7a0949e586cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.json +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala index e5b7ae0bd228..698423948f91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptionsInRead} +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala index 5c41bbd93198..a64b78d3c830 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JSONOptionsInRead import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.json.JsonDataSource -import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala index bb3c0366bdc2..be53b1b1676f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala index bbdd3ae69222..9bb615528fc5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 1ea80d2ba5fb..8665af33b976 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index ec923797e269..3d6206128cab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -29,13 +29,11 @@ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index a4fb03405d16..40784516a6f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -20,10 +20,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 458b98c627be..8d1d4ec45915 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConverters._ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcFilters import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala index 3fe433861a3c..f2e4b88e9f1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala index 0b6d5a960374..8cb6186c12ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.parquet +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index a0f19c3dd2eb..ded145f7de94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -33,12 +33,12 @@ 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.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} import org.apache.spark.sql.execution.datasources.parquet._ import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index a67aa3b92ce8..7e6ea41cf0b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -21,13 +21,12 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetWriteSupport} import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala index 4b8b434af88e..87db00077e79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution.datasources.v2.parquet import scala.collection.JavaConverters._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, SparkToParquetSchemaConverter} import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala index dce851dbcd33..2ad64b1aa524 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala index f6aa1e9c898b..049c717effa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.text +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala index 878888711188..0cd184da6ef8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala @@ -21,11 +21,11 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, HadoopFileWholeTextReader, PartitionedFile} import org.apache.spark.sql.execution.datasources.text.TextOptions import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala index 89b0511442d4..a2c42db59d7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.text.TextOptions import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala index fbe5e1688b83..b2b518c12b01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2.text import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala index b8cb61a6c646..87bfa84985e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2.text import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala index c9c2ebc875f2..7eb9768e483d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.SparkEnv import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset +import org.apache.spark.sql.connector.read.streaming.PartitionOffset case class ContinuousRecordPartitionOffset(partitionId: Int, offset: Int) extends PartitionOffset case class GetRecord(offset: ContinuousRecordPartitionOffset) 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 e7eb2cb558cd..5fe1f92e396c 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 @@ -24,12 +24,12 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchStream, WriteToMicroBatchDataSource} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index b6fa2e9dc361..357c049aa18f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -22,9 +22,9 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, StreamingAggregationStateManager} import org.apache.spark.sql.internal.SQLConf.{FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, _} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala index b40426aff0e7..f6543c3e4c4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} /** * This class is used to log offsets to persistent files in HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 6cb75083d0c0..204c6a4febfd 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 @@ -27,10 +27,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, SparkDataStream} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2Relation, StreamWriterCommitProgress} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, SparkDataStream} import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent import org.apache.spark.util.Clock diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala index 02fed50485b9..84f0961e4af1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.execution.streaming import org.json4s.DefaultFormats import org.json4s.jackson.Serialization -import org.apache.spark.sql.sources.v2 +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} case class RateStreamOffset(partitionToValueAndRunTimeMs: Map[Int, ValueRunTimeMsPair]) - extends v2.reader.streaming.Offset { + extends OffsetV2 { implicit val defaultFormats: DefaultFormats = DefaultFormats override val json = Serialization.write(partitionToValueAndRunTimeMs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 190325fb7ec2..36c7796ec439 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import java.util import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{Table, TableCapability} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 7f66d0b055cc..6d51d7dc4417 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 7c1f6ca42c1f..debf1fad5730 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -22,7 +22,7 @@ import java.nio.channels.ClosedByInterruptException import java.util.UUID import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} import java.util.concurrent.atomic.AtomicReference -import java.util.concurrent.locks.{Condition, ReentrantLock} +import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} @@ -36,14 +36,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.connector.write.SupportsTruncate +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} -import org.apache.spark.sql.sources.v2.writer.SupportsTruncate -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 8783eaa0e68b..c6634f51d038 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.execution.streaming import scala.collection.{immutable, GenTraversableOnce} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} - +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} /** * A helper class that looks like a Map[Source, Offset]. 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 142b6e7d1806..5858c54ce554 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 @@ -23,10 +23,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.connector.catalog.{Table, TableProvider} +import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.sources.v2.{Table, TableProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.util.CaseInsensitiveStringMap object StreamingRelation { 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 9ae39c79c515..20eb7ae5a6d9 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 @@ -22,11 +22,11 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql._ +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.write.{SupportsTruncate, WriteBuilder} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer.{SupportsTruncate, WriteBuilder} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap 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 b68f67e0b22d..5ee27c71aa73 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 @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.streaming.continuous 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.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.util.NextIterator 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 f6d156ded766..8c7371e75b53 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 @@ -28,12 +28,11 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, TableCapability} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, Offset => OffsetV2, PartitionOffset} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} -import org.apache.spark.sql.sources.v2 -import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock @@ -340,7 +339,7 @@ class ContinuousExecution( val offset = sources(0).deserializeOffset(offsetLog.get(epoch).get.offsets(0).get.json) committedOffsets ++= Seq(sources(0) -> offset) - sources(0).commit(offset.asInstanceOf[v2.reader.streaming.Offset]) + sources(0).commit(offset.asInstanceOf[OffsetV2]) } else { return } 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 65c5fc63c2f4..dff2fa69e42f 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 @@ -26,7 +26,7 @@ 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.connector.read.streaming.{ContinuousPartitionReader, PartitionOffset} import org.apache.spark.sql.types.StructType import org.apache.spark.util.ThreadUtils 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 e1b7a8fc283d..e66a1fe48a2e 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 @@ -22,9 +22,9 @@ 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.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ case class RateStreamPartitionOffset( partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset 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 2263b42870a6..aaeb13defd34 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 @@ -32,10 +32,10 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{Offset => _, _} import org.apache.spark.sql.execution.streaming.sources.TextSocketReader -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.RpcUtils 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 a08411d746ab..6fcd6ba64f65 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,8 @@ 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.connector.write.DataWriter +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory import org.apache.spark.util.Utils /** 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 decf524f7167..dbddab2e9acd 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.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.util.RpcUtils private[continuous] sealed trait EpochCoordinatorMessage extends Serializable 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 54f484c4adae..cecb2843fc3b 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,7 +19,7 @@ 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.StreamingWrite +import org.apache.spark.sql.connector.write.streaming.StreamingWrite /** * The logical plan for writing data in a continuous stream. 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 2f3af6a6544c..d4e522562e91 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 @@ -24,9 +24,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * The physical plan for writing data into a continuous processing [[StreamingWrite]]. 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 df149552dfb3..26cf1fb8def5 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 @@ -31,10 +31,10 @@ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index dbe242784986..6afb811a4d99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -20,8 +20,8 @@ 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.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap 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 41eaf84b7f9e..8f89ccd69876 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 @@ -29,9 +29,9 @@ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.{Encoder, SQLContext} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{Offset => _, _} -import org.apache.spark.sql.sources.v2.reader.InputPartition -import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.util.RpcUtils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 838c7d497e35..bae7fa7d0735 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -26,10 +26,10 @@ 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.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.python.PythonForeachWriter -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index f3951897ea74..5f12832cd255 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} /** * A [[BatchWrite]] used to hook V2 stream writers into a microbatch plan. It implements 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 fd4cb444ce58..53f56edc2768 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,8 +21,8 @@ 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.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory /** * A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala index 6a66f52c8f73..fb46f7668268 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming.sources -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} // A special `MicroBatchStream` that can get latestOffset with a start offset. trait RateControlMicroBatchStream extends MicroBatchStream { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala index 156ba95ab973..eb6baf698a5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala @@ -27,9 +27,9 @@ 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.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{ManualClock, SystemClock} 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 f61e9dbecd4e..3f7b0377f1ea 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 @@ -23,11 +23,11 @@ import scala.collection.JavaConverters._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousStream import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala index 25e9af2bc292..97a657683251 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala @@ -28,9 +28,9 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming.LongOffset -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.unsafe.types.UTF8String /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala index 0f807e235661..fae3cb765c0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala @@ -26,11 +26,11 @@ import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.sql._ +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousStream import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala index a3f58fa966fe..ef1115e6d9e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * The logical plan for writing data to a micro-batch stream. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index de8d00d4ac34..51ab5ce3578a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -32,10 +32,10 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Attribute 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.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.streaming.Sink -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index a83a0f51ecf1..92e255ae8aaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -24,12 +24,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Unstable import org.apache.spark.sql._ -import org.apache.spark.sql.catalog.v2.CatalogManager import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListener} 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 23a84cbd0dc0..4a6516d325dd 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 @@ -24,13 +24,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.connector.catalog.{SupportsRead, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap 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 36104d7a7044..74170b1b5d77 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 @@ -25,13 +25,13 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.connector.catalog.{SupportsWrite, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ -import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** 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 976595616bd2..9abe38dfda0b 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 @@ -31,12 +31,12 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} import org.apache.spark.sql.execution.streaming._ 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.{SupportsWrite, Table} import org.apache.spark.util.{Clock, SystemClock, Utils} /** 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/connector/JavaAdvancedDataSourceV2.java similarity index 96% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java index 255a9f887878..9386ab51d64f 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/connector/JavaAdvancedDataSourceV2.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import java.util.*; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.*; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; 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/connector/JavaColumnarDataSourceV2.java similarity index 90% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java index 699859cfaebe..76da45e182b3 100644 --- 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/connector/JavaColumnarDataSourceV2.java @@ -15,15 +15,18 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.apache.spark.sql.vectorized.ColumnVector; 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/connector/JavaPartitionAwareDataSource.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java index 391af5a306a1..fbbc457b2945 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/connector/JavaPartitionAwareDataSource.java @@ -15,21 +15,21 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import java.util.Arrays; -import org.apache.spark.sql.catalog.v2.expressions.Expressions; -import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -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.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.*; +import org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution; +import org.apache.spark.sql.connector.read.partitioning.Distribution; +import org.apache.spark.sql.connector.read.partitioning.Partitioning; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaPartitionAwareDataSource implements TableProvider { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java similarity index 90% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java index 438f489a3eea..d612441201e6 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.connector.read.InputPartition; class JavaRangeInputPartition implements InputPartition { int start; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java similarity index 82% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java index f3755e18b58d..49438fe668d5 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.util.OptionalLong; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; -import org.apache.spark.sql.sources.v2.reader.Statistics; -import org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsReportStatistics; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaReportStatisticsDataSource implements TableProvider { 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/connector/JavaSchemaRequiredDataSource.java similarity index 87% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java index 3800a94f8889..2181887ae54e 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/connector/JavaSchemaRequiredDataSource.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java index 64663d5db4be..97b00477e176 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.util.Arrays; import java.util.HashSet; import java.util.Set; -import org.apache.spark.sql.sources.v2.SupportsRead; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.connector.catalog.SupportsRead; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; import org.apache.spark.sql.types.StructType; abstract class JavaSimpleBatchTable implements Table, SupportsRead { 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/connector/JavaSimpleDataSourceV2.java similarity index 84% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java index 7474f36c97f7..8b6d71b986ff 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/connector/JavaSimpleDataSourceV2.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaSimpleDataSourceV2 implements TableProvider { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java index 740279033c41..0c702031a939 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; class JavaSimpleReaderFactory implements PartitionReaderFactory { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java similarity index 81% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java index 217e66950d14..7cbba0042092 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.reader.Batch; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; -import org.apache.spark.sql.sources.v2.reader.Scan; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; abstract class JavaSimpleScanBuilder implements ScanBuilder, Scan, Batch { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 4b7ee384b4c1..03cefa0d2e77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala similarity index 94% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 629e825ec68d..207ece83759e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -15,21 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.InMemoryTable +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG -import org.apache.spark.sql.sources.v2.utils.TestV2SessionCatalogBase import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,8 +35,6 @@ class DataSourceV2DataFrameSessionCatalogSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) with SessionCatalogTest[InMemoryTable, InMemoryTableSessionCatalog] { - import testImplicits._ - override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = { val dfw = insert.write.format(v2Format) if (mode != null) { @@ -135,7 +130,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable } } -private[v2] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalogBase[T]] +private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalogBase[T]] extends QueryTest with SharedSparkSession with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala similarity index 97% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index abccb5cec675..cd811bb7afb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, Row, SaveMode} -import org.apache.spark.sql.connector.InMemoryTableCatalog class DataSourceV2DataFrameSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala similarity index 89% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index 8df65c14a8d9..e1a5dbe3351e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, SaveMode} -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.connector.InMemoryTable -import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} class DataSourceV2SQLSessionCatalogSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4bfbefef44bb..ef484ce6ac36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -15,17 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.{BasicInMemoryTableCatalog, InMemoryTable, InMemoryTableCatalog, StagingInMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,7 +31,7 @@ class DataSourceV2SQLSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true) with AlterTableTests { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ private val v2Source = classOf[FakeV2Provider].getName override protected val v2Format = v2Source 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/connector/DataSourceV2Suite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 8f7dbe8d13c3..00a8b430d33c 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/connector/DataSourceV2Suite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.io.File import java.util @@ -23,19 +23,20 @@ import java.util.OptionalLong import scala.collection.JavaConverters._ -import test.org.apache.spark.sql.sources.v2._ +import test.org.apache.spark.sql.connector._ import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.{Filter, GreaterThan} -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala similarity index 95% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala index 0b1e3b5fb076..01fcced5b12a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.connector.catalog.SessionConfigSupport import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index 26f941244f5c..cd804adfa213 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -14,21 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution} import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 5b5382e5ca93..0fd6cf1b6746 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.scalatest.BeforeAndAfter @@ -175,7 +175,7 @@ abstract class InsertIntoTests( } } -private[v2] trait InsertIntoSQLOnlyTests +trait InsertIntoSQLOnlyTests extends QueryTest with SharedSparkSession with BeforeAndAfter { 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/connector/SimpleWritableDataSource.scala similarity index 94% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index c9d2f1eef24b..22d3750022c5 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/connector/SimpleWritableDataSource.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.io.{BufferedReader, InputStreamReader, IOException} import java.util @@ -27,9 +27,10 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, ScanBuilder} +import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -190,7 +191,7 @@ class CSVReaderFactory(conf: SerializableConfiguration) } } -private[v2] object SimpleCounter { +private[connector] object SimpleCounter { private var count: Int = 0 def increaseCounter: Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index ab4783600170..39f4085a9baf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util @@ -24,12 +24,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LeafNode, OverwriteByExpression, OverwritePartitionsDynamic, Union} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.{Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, TableCapabilityCheck} import org.apache.spark.sql.execution.streaming.{Offset, Source, StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_WRITE, CONTINUOUS_READ, MICRO_BATCH_READ, OVERWRITE_BY_FILTER, OVERWRITE_DYNAMIC, TRUNCATE} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala similarity index 88% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index b25eab154626..d03294cb4067 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.utils +package org.apache.spark.sql.connector import java.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{DelegatingCatalogExtension, Identifier} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** @@ -32,7 +31,7 @@ import org.apache.spark.sql.types.StructType * for testing DDL as well as write operations (through df.write.saveAsTable, df.write.insertInto * and SQL). */ -private[v2] abstract class TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { +private[connector] trait TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { protected val tables: util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala similarity index 92% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 9002775bce21..7cd6ba21b56e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util @@ -24,13 +24,11 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform, Transform} -import org.apache.spark.sql.connector.InMemoryTable -import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} +import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode, SparkSession} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.sources.{DataSourceRegister, Filter, InsertableRelation} -import org.apache.spark.sql.sources.v2.utils.TestV2SessionCatalogBase -import org.apache.spark.sql.sources.v2.writer.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index e2cee593af03..7110f13e2ead 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -25,16 +25,15 @@ import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} -import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.{InMemoryTableCatalog, InMemoryTableProvider} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG -import org.apache.spark.sql.sources.v2.InMemoryTableProvider import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala index ad0dfadacca1..7f4bbcf97b53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.{QueryTest, SparkSession} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index a309152b2614..6bd6e0d89a67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -24,17 +24,16 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, NamespaceChange, TableChange} import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.connector.catalog.{Catalogs, Identifier, NamespaceChange, TableChange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class V2SessionCatalogBaseSuite extends SparkFunSuite with SharedSparkSession with BeforeAndAfter { +abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeAndAfter { val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] val schema: StructType = new StructType() @@ -54,7 +53,7 @@ class V2SessionCatalogBaseSuite extends SparkFunSuite with SharedSparkSession wi class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override protected def beforeAll(): Unit = { super.beforeAll() @@ -752,7 +751,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ def checkMetadata( expected: scala.collection.Map[String, String], 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 ef88598fcb11..6440e69e2ec2 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 @@ -24,12 +24,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation 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.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ManualClock 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 e1284ea03267..b1e645983815 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 @@ -29,12 +29,12 @@ import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation 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.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ 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 7914a713f0ba..e54a5370d351 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 @@ -37,12 +37,12 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.AllTuples import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochCoordinatorRef, IncrementAndGetEpoch} import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StateStore -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{Clock, SystemClock, Utils} 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 d96404863a25..d84dc62a696b 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 @@ -30,9 +30,9 @@ import org.scalatest.concurrent.Waiters.Waiter import org.apache.spark.SparkException import org.apache.spark.scheduler._ import org.apache.spark.sql.{Encoder, SparkSession} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.util.JsonProtocol diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 3ad893f871c9..4a674b647917 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 @@ -35,13 +35,13 @@ import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -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 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 bad22590807a..5d6050dbda0f 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 @@ -26,9 +26,9 @@ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{DataType, IntegerType, StructType} 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 e3498db4194e..48998f5d6c5e 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 @@ -26,11 +26,11 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.LocalSparkSession +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf.CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.test.TestSparkSession class EpochCoordinatorSuite 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 4db605ee1b23..e9d148c38e6c 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 @@ -23,16 +23,16 @@ import java.util.Collections import scala.collection.JavaConverters._ import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsRead, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, MicroBatchStream, Offset, PartitionOffset} +import org.apache.spark.sql.connector.write.{WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} 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.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.sources.v2.writer.{WriteBuilder, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamTest, Trigger} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap