diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala index 9a7c9f2c6c6a7..02568aa89eb1d 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.connector.metric.CustomTaskMetric import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.execution.streaming.{MicroBatchExecution, StreamExecution} +import org.apache.spark.sql.execution.streaming.runtime.{MicroBatchExecution, StreamExecution} import org.apache.spark.sql.kafka010.consumer.KafkaDataConsumer /** A [[InputPartition]] for reading Kafka data in a batch based streaming query. */ diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceInitialOffsetWriter.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceInitialOffsetWriter.scala index 3854b8a86f16c..cef712fafd888 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceInitialOffsetWriter.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceInitialOffsetWriter.scala @@ -21,7 +21,8 @@ import java.io._ import java.nio.charset.StandardCharsets import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} +import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog +import org.apache.spark.sql.execution.streaming.runtime.SerializedOffset import org.apache.spark.util.Utils /** A version of [[HDFSMetadataLog]] specialized for saving the initial offsets. */ diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 8790a77e5d895..d2c0c5d6fba85 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -21,7 +21,8 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.sql.connector.read.streaming import org.apache.spark.sql.connector.read.streaming.PartitionOffset -import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.Offset +import org.apache.spark.sql.execution.streaming.runtime.SerializedOffset /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala index 4b6a5b899fc8e..2dff58ff87875 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -22,8 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.SparkContext import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec -import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.streaming.Trigger import org.apache.spark.sql.test.TestSparkSession diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala index 702bd4f6ebdad..a55b6e0068519 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala @@ -26,7 +26,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_PLAINTEXT import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.{KEYTAB, PRINCIPAL} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.{OutputMode, StreamTest} import org.apache.spark.sql.test.SharedSparkSession diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index a365c61fea577..2ef6c992ad238 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -42,8 +42,10 @@ import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2ScanRelation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, ASYNC_PROGRESS_TRACKING_ENABLED} +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeq import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution +import org.apache.spark.sql.execution.streaming.runtime.{MicroBatchExecution, StreamExecution, StreamingExecutionRelation} +import org.apache.spark.sql.execution.streaming.runtime.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, ASYNC_PROGRESS_TRACKING_ENABLED} import org.apache.spark.sql.functions.{count, expr, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.kafka010.KafkaSourceProvider._ diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 82edba59995ec..a444128cb2c0b 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -32,7 +32,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkContext, TestUtils} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} -import org.apache.spark.sql.execution.streaming.{MemoryStream, MemoryStreamBase} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, MemoryStreamBase} import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala index 553ab42b9c8b1..18330feac861c 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.kafka010 import java.io.File -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.checkpointing.{OffsetSeq, OffsetSeqLog} +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.streaming.OffsetSuite import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils diff --git a/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala b/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala index 4fe5bb007bf97..e64ba7882b918 100644 --- a/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala +++ b/hadoop-cloud/src/main/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManager.scala @@ -25,8 +25,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.spark.internal.{Logging, LogKeys} -import org.apache.spark.sql.execution.streaming.AbstractFileContextBasedCheckpointFileManager -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.checkpointing.AbstractFileContextBasedCheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream class AbortableStreamBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) extends AbstractFileContextBasedCheckpointFileManager(path, hadoopConf) with Logging { diff --git a/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala b/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala index 0dbc650fc8c73..0b7e65ff78d98 100644 --- a/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala +++ b/hadoop-cloud/src/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala @@ -27,8 +27,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.internal.Logging import org.apache.spark.internal.io.cloud.abortable.AbortableFileSystem -import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManagerTests +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager class AbortableStreamBasedCheckpointFileManagerSuite extends CheckpointFileManagerTests with Logging { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala index 14faa81f7bffa..ade7823326f42 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.StreamTest class VectorSizeHintSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index 18cce169b4ce9..a35b19b2816e4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -28,7 +28,7 @@ import org.apache.spark.ml.attribute._ import org.apache.spark.ml.classification._ import org.apache.spark.ml.linalg._ import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 16066e74d17f5..3913416c0dde3 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -77,8 +77,8 @@ import org.apache.spark.sql.execution.datasources.v2.python.UserDefinedPythonDat import org.apache.spark.sql.execution.python.{UserDefinedPythonFunction, UserDefinedPythonTableFunction} import org.apache.spark.sql.execution.python.streaming.PythonForeachWriter import org.apache.spark.sql.execution.stat.StatFunctions -import org.apache.spark.sql.execution.streaming.GroupStateImpl.groupStateTimeoutFromString -import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.groupStateTimeoutFromString +import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryWrapper import org.apache.spark.sql.expressions.{Aggregator, ReduceAggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, StatefulProcessor, StatefulProcessorWithInitialState, StreamingQuery, StreamingQueryListener, StreamingQueryProgress, Trigger} import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala index 07208ca7760db..a7b3385a27820 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataSource.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.execution.datasources.v2.python.PythonDataSourceV2 -import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation import org.apache.spark.sql.sources.StreamSourceProvider 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/classic/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala index 0fa6e91e21459..0b1da71be9ede 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala @@ -444,8 +444,8 @@ class RelationalGroupedDataset protected[sql]( stateStructType: StructType, outputModeStr: String, timeoutConfStr: String): DataFrame = { - val timeoutConf = org.apache.spark.sql.execution.streaming - .GroupStateImpl.groupStateTimeoutFromString(timeoutConfStr) + val timeoutConf = org.apache.spark.sql.execution.streaming.operators.stateful. + flatmapgroupswithstate.GroupStateImpl.groupStateTimeoutFromString(timeoutConfStr) val outputMode = InternalOutputModes(outputModeStr) if (outputMode != OutputMode.Append && outputMode != OutputMode.Update) { throw new IllegalArgumentException("The output mode of function should be append or update") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala index 5d2b74411b9c5..0470f3b20ecc8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingQueryManager.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, SupportsWrite, Table, import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution +import org.apache.spark.sql.execution.streaming.runtime.{AsyncProgressTrackingMicroBatchExecution, MicroBatchExecution, StreamingQueryListenerBus, StreamingQueryWrapper} 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 86b67953750d2..6109f7ad800dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -44,7 +44,8 @@ import org.apache.spark.sql.execution.bucketing.{CoalesceBucketsInJoin, DisableU import org.apache.spark.sql.execution.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.execution.reuse.ReuseExchangeAndSubquery -import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata, WatermarkPropagator} +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata +import org.apache.spark.sql.execution.streaming.runtime.{IncrementalExecution, WatermarkPropagator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.scripting.SqlScriptingExecution import org.apache.spark.sql.streaming.OutputMode diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index e23245adacfc6..f76bc911bef8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -40,7 +40,11 @@ import org.apache.spark.sql.execution.datasources.{WriteFiles, WriteFilesExec} import org.apache.spark.sql.execution.exchange.{REBALANCE_PARTITIONS_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REPARTITION_BY_COL, REPARTITION_BY_NUM, ShuffleExchangeExec} import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec} -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StreamingDeduplicateExec, StreamingDeduplicateWithinWatermarkExec, StreamingGlobalLimitExec, StreamingLocalLimitExec, UpdateEventTimeColumnExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExec +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinExec +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateExec +import org.apache.spark.sql.execution.streaming.runtime.{StreamingExecutionRelation, StreamingRelation, StreamingRelationExec} import org.apache.spark.sql.execution.streaming.sources.MemoryPlan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index de1b83c16ac97..58055fa6129a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.operators.stateful._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils import org.apache.spark.util.collection.{Utils => CUtils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala index 5391d5807597c..7ae00467a0114 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, PartialMerge} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{ExplainUtils, PartitioningPreservingUnaryExecNode, UnaryExecNode} -import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorPartitioning /** * Holds common logic for aggregate operators diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala index c3786a5338d77..b0b6d45620ac9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorPartitioning /** * This node updates the session window spec of each input rows via analyzing neighbor rows and diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index ee21d7e970dfd..0ea52f3dccb84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.{CommandExecutionMode, ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.IncrementalExecution +import org.apache.spark.sql.execution.streaming.runtime.IncrementalExecution import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ 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 1f9abf6e3a965..3816a446a57c7 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 @@ -49,7 +49,9 @@ import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.python.PythonDataSourceV2 import org.apache.spark.sql.execution.datasources.xml.XmlFileFormat -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.execution.streaming.runtime._ +import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala index ec8182304f791..3e6dc6255d7f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolver.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.UnresolvedCatalogRelation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.classic.SparkSession -import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation /** * The [[DataSourceResolver]] is a [[Resolver]] extension that resolves nodes defined in the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 21acc99db265a..15b34457923fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -54,7 +54,7 @@ import org.apache.spark.sql.execution import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, PushedDownOperators} -import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources import org.apache.spark.sql.sources._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index 03ddc7d609346..2d68faa3ff526 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -30,7 +30,7 @@ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.FileSourceOptions import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.execution.streaming.FileStreamSink +import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink import org.apache.spark.sql.types.StructType import org.apache.spark.util.HadoopFSUtils 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 863104da80c2e..0af728c1958d4 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 @@ -29,7 +29,8 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex} +import org.apache.spark.sql.execution.streaming.runtime.MetadataLogFileIndex +import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink 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/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index e332c6b8014a6..072dcc995f41c 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation import org.apache.spark.sql.types.BooleanType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala index 9595e1bb71a13..8b35822e83fac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala @@ -33,10 +33,15 @@ import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.{J import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues.JoinSideValues import org.apache.spark.sql.execution.datasources.v2.state.metadata.{StateMetadataPartitionReader, StateMetadataTableEntry} import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.{OffsetSeqMetadata, StatefulOperatorsUtils, StreamingQueryCheckpointMetadata, TimerStateUtils, TransformWithStateOperatorProperties, TransformWithStateVariableInfo} -import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_STATE -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} -import org.apache.spark.sql.execution.streaming.state.{InMemoryStateSchemaProvider, KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, StateSchemaCompatibilityChecker, StateSchemaMetadata, StateSchemaProvider, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorsUtils +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStateOperatorProperties, TransformWithStateVariableInfo} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.TimerStateUtils +import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_STATE +import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryCheckpointMetadata +import org.apache.spark.sql.execution.streaming.state.{InMemoryStateSchemaProvider, KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, StateSchemaCompatibilityChecker, StateSchemaMetadata, StateSchemaProvider, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreProviderId} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.streaming.TimeMode import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala index 564141551b289..6402eba868ef0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.{StateVariableType, TransformWithStateVariableInfo} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateVariableType, TransformWithStateVariableInfo} import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.execution.streaming.state.RecordType.{getRecordTypeAsString, RecordType} import org.apache.spark.sql.types.{NullType, StructField, StructType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala index 9adabc7096bb0..a1bd88f40740a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala @@ -25,8 +25,8 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} -import org.apache.spark.sql.execution.streaming.TransformWithStateVariableInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableInfo import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, StateSchemaProvider, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors} 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/state/StateTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala index 96614f0613c9d..dfa378e8a5972 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataC import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.TransformWithStateVariableInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableInfo import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, StateSchemaProvider, StateStoreColFamilySchema, StateStoreConf} 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/state/StreamStreamJoinStateHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala index 3abd1924f5430..5cb38022159cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStateHelper.scala @@ -22,9 +22,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.streaming.CheckpointFileManager -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide} -import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager +import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreId, StateStoreProviderId} import org.apache.spark.sql.types.{BooleanType, StructType} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala index 82415b9f30c54..f1415865db246 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala @@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.JoinSideValues import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide} -import org.apache.spark.sql.execution.streaming.state.{JoinStateManagerStoreGenerator, StateStoreConf, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.join.{JoinStateManagerStoreGenerator, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{JoinSide, LeftSide, RightSide} +import org.apache.spark.sql.execution.streaming.state.StateStoreConf import org.apache.spark.sql.types.{BooleanType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala index ae4483154e25b..31e6ac30a5984 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/metadata/StateMetadataSource.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceErrors import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.PATH -import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, OperatorStateMetadataReader, OperatorStateMetadataUtils, OperatorStateMetadataV1, OperatorStateMetadataV2} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StringType, StructType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala index 84eab3356c204..52df016791d48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/utils/SchemaUtil.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.execution.datasources.v2.state.{StateDataSourceErrors, StateSourceOptions} -import org.apache.spark.sql.execution.streaming.{StateVariableType, TransformWithStateVariableInfo} -import org.apache.spark.sql.execution.streaming.StateVariableType._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateVariableType, TransformWithStateVariableInfo} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType._ import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStoreColFamilySchema, UnsafeRowPair} import org.apache.spark.sql.types.{ArrayType, DataType, IntegerType, LongType, MapType, StringType, StructType} import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 0b97c8ebab815..b408f2823e16a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.StringConcat import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} -import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.runtime.{StreamExecution, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index ca7836992aacb..015872d98c372 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.python.BatchIterator import org.apache.spark.sql.execution.r.ArrowRRunner -import org.apache.spark.sql.execution.streaming.GroupStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.GroupStateTimeout import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala index 1ba219837f43f..b6f6a4cbc30b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStatePythonRunner.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.{PythonArrowInput, PythonArrowOutput, PythonUDFRunner} import org.apache.spark.sql.execution.python.streaming.ApplyInPandasWithStatePythonRunner.{COUNT_COLUMN_SCHEMA_FROM_PYTHON_WORKER, InType, OutType, OutTypeForState, STATE_METADATA_SCHEMA_FROM_PYTHON_WORKER} import org.apache.spark.sql.execution.python.streaming.ApplyInPandasWithStateWriter.STATE_METADATA_SCHEMA -import org.apache.spark.sql.execution.streaming.GroupStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala index 6bfa5440db37f..f55ca749112fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/ApplyInPandasWithStateWriter.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.execution.arrow.ArrowWriter.createFieldWriter -import org.apache.spark.sql.execution.streaming.GroupStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl import org.apache.spark.sql.types.{BinaryType, BooleanType, IntegerType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala index e53bdae813d26..5b9b95ef413aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/FlatMapGroupsInPandasWithStateExec.scala @@ -28,9 +28,10 @@ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.python.ArrowPythonRunner import org.apache.spark.sql.execution.python.PandasGroupUtils.resolveArgOffsets -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP -import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper.StateData +import org.apache.spark.sql.execution.streaming.operators.stateful._ +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.{FlatMapGroupsWithStateExecBase, GroupStateImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper.StateData +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala index 4cb9243132485..1b967af38b6db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkExec.scala @@ -35,8 +35,10 @@ import org.apache.spark.sql.execution.{CoGroupedIterator, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.ArrowPythonRunner import org.apache.spark.sql.execution.python.PandasGroupUtils.{executePython, groupAndProject, resolveArgOffsets} -import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulOperatorStateInfo, StatefulOperatorsUtils, StatefulProcessorHandleImpl, TransformWithStateExecBase, TransformWithStateVariableInfo} -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOperatorsUtils} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStateExecBase, TransformWithStateVariableInfo} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl} import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StateSchemaValidationResult, StateStore, StateStoreColFamilySchema, StateStoreConf, StateStoreId, StateStoreOps, StateStoreProvider, StateStoreProviderId} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, TimeMode} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala index dffdaca1b835e..51dc179c901ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkPythonRunner.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.{BasicPythonArrowOutput, PythonArrowInput, PythonUDFRunner} import org.apache.spark.sql.execution.python.streaming.TransformWithStateInPySparkPythonRunner.{GroupedInType, InType} -import org.apache.spark.sql.execution.streaming.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, StatefulProcessorHandleImpl} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala index 4b50cee44e737..4edeae132b47a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServer.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.api.python.PythonSQLUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleImplBase, StatefulProcessorHandleState, StateVariableType} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleImplBase, StatefulProcessorHandleState} import org.apache.spark.sql.execution.streaming.state.StateMessage.{HandleState, ImplicitGroupingKeyRequest, ListStateCall, MapStateCall, StatefulProcessorCall, StateRequest, StateResponse, StateResponseWithLongTypeVal, StateResponseWithMapIterator, StateResponseWithMapKeysOrValues, StateResponseWithStringTypeVal, StateResponseWithTimer, StateVariableRequest, TimerInfo, TimerRequest, TimerStateCallCommand, TimerValueRequest, UtilsRequest, ValueStateCall} import org.apache.spark.sql.execution.streaming.state.StateMessage.KeyAndValuePair import org.apache.spark.sql.execution.streaming.state.StateMessage.StateResponseWithListGet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala index 3cb9bfa24a536..5dc7d9733dcdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/streaming/benchmark/BenchmarkTransformWithStateInPySparkStateServer.scala @@ -33,9 +33,7 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.python.streaming.TransformWithStateInPySparkStateServer -import org.apache.spark.sql.execution.streaming.ImplicitGroupingKeyTracker -import org.apache.spark.sql.execution.streaming.QueryInfoImpl -import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleImplBase +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, QueryInfoImpl, StatefulProcessorHandleImplBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.ListState import org.apache.spark.sql.streaming.MapState diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index a628126990305..66e90ec689131 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.LogKeys.{BATCH_ID, PATH} import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSinkLog, SinkFileStatus} /** * A [[FileCommitProtocol]] that tracks the list of valid files in a manifest file, used in diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/Sink.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/Source.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala index dce2ea9f14cee..116ea18326ef0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncCommitLog.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io.OutputStream import java.util.concurrent.{CompletableFuture, ConcurrentLinkedDeque, ThreadPoolExecutor} import scala.jdk.CollectionConverters._ -import org.apache.spark.internal.{LogKeys} +import org.apache.spark.internal.LogKeys import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala index cd31ae4baa317..18d18e61da475 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/AsyncOffsetSeqLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io.OutputStream import java.util.concurrent._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala index afc4a901348d0..3d07483ba4dd1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CheckpointFileManager.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io.{FileNotFoundException, OutputStream} import java.lang.reflect.InvocationTargetException @@ -30,7 +30,8 @@ import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{FINAL_PATH, PATH, TEMP_PATH} import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.RenameHelperMethods +import org.apache.spark.sql.execution.streaming.StreamingErrors +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.RenameHelperMethods import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -50,7 +51,7 @@ import org.apache.spark.util.Utils */ trait CheckpointFileManager { - import org.apache.spark.sql.execution.streaming.CheckpointFileManager._ + import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager._ /** * Create a file and make its contents available atomically after the output stream is closed. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala index f501b62b8a718..6892b6b535cf9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/CommitLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io.{InputStream, OutputStream} import java.nio.charset.StandardCharsets._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala index 62b9c665e70b9..6d35b1a8f8c00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/HDFSMetadataLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io._ import java.nio.charset.StandardCharsets diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala index 4083d5d897afc..18c04e0b9b5db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing /** * A general MetadataLog that supports the following features: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala index 854e36b5304a9..8d832c21287a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/MetadataVersionUtil.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import org.apache.spark.sql.errors.QueryExecutionErrors diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala index 24d56672ccad7..c1c3c379719a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeq.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization @@ -25,7 +25,10 @@ import org.apache.spark.internal.LogKeys.{CONFIG, DEFAULT_VALUE, NEW_VALUE, OLD_ import org.apache.spark.io.CompressionCodec 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, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingAggregationStateManager +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager +import org.apache.spark.sql.execution.streaming.runtime.{MultipleWatermarkPolicy, StreamProgress} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala index 7e490ef4cd53d..816563b3f09fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/checkpointing/OffsetSeqLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.checkpointing import java.io.{InputStream, OutputStream} @@ -25,6 +25,7 @@ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} +import org.apache.spark.sql.execution.streaming.runtime.SerializedOffset /** * 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/sinks/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/console.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 43381fe034af4..51cd457fbc856 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 @@ -39,6 +39,8 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation} import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitMetadata, OffsetSeq} +import org.apache.spark.sql.execution.streaming.runtime.{AcceptsLatestSeenOffsetHandler, ACTIVE, ContinuousExecutionContext, IncrementalExecution, ProcessingTimeExecutor, RECONFIGURING, State, StreamExecution, StreamExecutionContext, TERMINATED, WatermarkPropagator} import org.apache.spark.sql.streaming.Trigger import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Clock 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 132d9a9d61e57..ef5eab951b605 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 @@ -24,7 +24,7 @@ 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.execution.streaming.runtime.{RateStreamOffset, ValueRunTimeMsPair} 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 8adde9d3ff15b..c23e38a13efee 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 @@ -38,7 +38,8 @@ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, 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.execution.streaming.{Offset => _, _} +import org.apache.spark.sql.execution.streaming.{Offset => _} +import org.apache.spark.sql.execution.streaming.runtime.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord} import org.apache.spark.sql.execution.streaming.sources.TextSocketReader 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/operators/stateful/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala index d25c4be0fb84a..03c8d70ec87f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/EventTimeWatermarkExec.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful import java.util.UUID diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala index a923ebd79889c..69577a1f74339 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/MergingSortWithSessionWindowStateIterator.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StreamingSessionWindowStateManager} +import org.apache.spark.sql.execution.streaming.state.ReadStateStore /** * This class technically does the merge sort between input rows and existing sessions in state, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala index 527349201574e..ed2941e3f2f4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StatefulOperatorPartitioning.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, StatefulOpClusteredDistribution} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala index 97feb9b579af9..c7f7f388010da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingAggregationStateManager.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.state +package org.apache.spark.sql.execution.streaming.operators.stateful import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, GenerateUnsafeRowJoiner} import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala index 8220045eaf212..a74b4aaf0da12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/StreamingSessionWindowStateManager.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.state +package org.apache.spark.sql.execution.streaming.operators.stateful import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, StateStore, UnsafeRowPair} import org.apache.spark.sql.types.{StructType, TimestampType} import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala index ebf08358c07bc..a0778fbfb6149 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate import java.util.concurrent.TimeUnit.NANOSECONDS @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.Distribution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._ +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorPartitioning, StatefulOperatorStateInfo, StateStoreWriter, WatermarkSupport} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._ import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode} import org.apache.spark.sql.streaming.GroupStateTimeout.NoTimeout diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala index b68c08b3ea5c6..280fcfc0ca1ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/FlatMapGroupsWithStateExecHelper.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.state +package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.execution.ObjectOperator -import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP +import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala index d4e93642b8164..1cab963802ca2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/flatmapgroupswithstate/GroupStateImpl.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate import java.sql.Date import java.util.concurrent.TimeUnit @@ -28,7 +28,7 @@ import org.apache.spark.api.java.Optional import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, NoTimeout, ProcessingTimeTimeout} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.GroupStateImpl._ +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupStateTimeout, TestGroupState} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala index 839d610550abb..f424d2892dfa1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinExec.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.join import java.util.concurrent.TimeUnit.NANOSECONDS @@ -29,9 +29,10 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._ +import org.apache.spark.sql.execution.streaming.operators.stateful.{SchemaValidationUtils, StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorStateInfo, StatefulOperatorsUtils, StatefulOpStateStoreCheckpointInfo, StateStoreWriter, WatermarkSupport} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._ +import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager.KeyToValuePair import org.apache.spark.sql.execution.streaming.state._ -import org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager.KeyToValuePair import org.apache.spark.sql.internal.{SessionState, SQLConf} import org.apache.spark.sql.types.StructType import org.apache.spark.util.{CompletionIterator, SerializableConfiguration} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala index 497e71070a09a..6f02a17efe340 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/StreamingSymmetricHashJoinHelper.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.join import scala.reflect.ClassTag @@ -26,7 +26,8 @@ import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, BoundReference, Expression, NamedExpression, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.streaming.WatermarkSupport.watermarkExpression +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, WatermarkSupport} +import org.apache.spark.sql.execution.streaming.operators.stateful.WatermarkSupport.watermarkExpression import org.apache.spark.sql.execution.streaming.state.{StateStoreCheckpointInfo, StateStoreCoordinatorRef, StateStoreProviderId} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala index 50e508da4b7db..4ba6dcced5335 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/join/SymmetricHashJoinStateManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.state +package org.apache.spark.sql.execution.streaming.operators.stateful.join import java.util.Locale @@ -30,9 +30,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, JoinedRow, Literal, SafeProjection, SpecificInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo -import org.apache.spark.sql.execution.streaming.StatefulOpStateStoreCheckpointInfo -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._ +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOpStateStoreCheckpointInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper._ +import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, NoPrefixKeyStateEncoderSpec, StateSchemaBroadcast, StateStore, StateStoreCheckpointInfo, StateStoreColFamilySchema, StateStoreConf, StateStoreErrors, StateStoreId, StateStoreMetrics, StateStoreProvider, StateStoreProviderId, SupportsFineGrainedReplay} import org.apache.spark.sql.types.{BooleanType, LongType, StructField, StructType} import org.apache.spark.util.NextIterator @@ -439,7 +440,7 @@ abstract class SymmetricHashJoinStateManager( * NOTE: this function is only intended for use in unit tests * to simulate null values. */ - private[state] def updateNumValuesTestOnly(key: UnsafeRow, numValues: Long): Unit = { + private[streaming] def updateNumValuesTestOnly(key: UnsafeRow, numValues: Long): Unit = { keyToNumValues.put(key, numValues) } @@ -528,7 +529,7 @@ abstract class SymmetricHashJoinStateManager( * Helper class for representing data returned by [[KeyWithIndexToValueStore]]. * Designed for object reuse. */ - private[state] class KeyAndNumValues(var key: UnsafeRow = null, var numValue: Long = 0) { + private[join] class KeyAndNumValues(var key: UnsafeRow = null, var numValue: Long = 0) { def withNew(newKey: UnsafeRow, newNumValues: Long): this.type = { this.key = newKey this.numValue = newNumValues @@ -595,7 +596,7 @@ abstract class SymmetricHashJoinStateManager( * Helper class for representing data returned by [[KeyWithIndexToValueStore]]. * Designed for object reuse. */ - private[state] class KeyWithIndexAndValue( + private[join] class KeyWithIndexAndValue( var key: UnsafeRow = null, var valueIndex: Long = -1, var value: UnsafeRow = null, @@ -1189,17 +1190,18 @@ object SymmetricHashJoinStateManager { } } - private[state] sealed trait StateStoreType + private[join] sealed trait StateStoreType - private[state] case object KeyToNumValuesType extends StateStoreType { + private[join] case object KeyToNumValuesType extends StateStoreType { override def toString(): String = "keyToNumValues" } - private[state] case object KeyWithIndexToValueType extends StateStoreType { + private[join] case object KeyWithIndexToValueType extends StateStoreType { override def toString(): String = "keyWithIndexToValue" } - private[state] def getStateStoreName(joinSide: JoinSide, storeType: StateStoreType): String = { + private[join] def getStateStoreName( + joinSide: JoinSide, storeType: StateStoreType): String = { s"$joinSide-$storeType" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala index 027b911262eac..637c4563f23fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/statefulOperators.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful import java.util.UUID import java.util.concurrent.TimeUnit._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala index 07fe0ae4357d0..6816be103f6e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/streamingLimits.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful import java.util.concurrent.TimeUnit.NANOSECONDS diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala index 4401f8cedff6b..7e25960daf33d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateStoreColumnFamilySchemaUtils.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate import scala.collection.mutable import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ -import org.apache.spark.sql.execution.streaming.TransformWithStateVariableUtils.getRowCounterCFName +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableUtils.getRowCounterCFName import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, RangeKeyScanStateEncoderSpec, StateStoreColFamilySchema} import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala index a2b7ee4ba7916..8ce300a40b43c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/StateTypesEncoderUtils.scala @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.StateTTL import org.apache.spark.sql.execution.streaming.state.StateStoreErrors import org.apache.spark.sql.types._ @@ -180,7 +182,7 @@ class CompositeKeyStateEncoder[K, V]( stateName: String, hasTtl: Boolean = false) extends StateTypesEncoder[V](keyEncoder, valEncoder, stateName, hasTtl) { - import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ + import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ /** Encoders */ private val userKeyExpressionEnc = encoderFor(userKeyEnc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala index db3b7841ac8ea..20e2c32015d84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate import java.util.UUID import java.util.concurrent.TimeUnit.NANOSECONDS @@ -28,7 +28,10 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeRow} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOperatorsUtils} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.StateStoreAwareZipPartitionsHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{DriverStatefulProcessorHandleImpl, ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala index df68b21e0bb9f..c2d24e735ab2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateExecBase.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.InternalRow @@ -22,6 +22,8 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, SortOrde import org.apache.spark.sql.catalyst.plans.logical.{EventTime, ProcessingTime} import org.apache.spark.sql.catalyst.plans.physical.Distribution import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorCustomMetric, StatefulOperatorCustomSumMetric, StatefulOperatorPartitioning, StateStoreWriter, WatermarkSupport} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadata, TransformWithStateUserFunctionException} import org.apache.spark.sql.streaming.{OutputMode, TimeMode} import org.apache.spark.sql.types.{BinaryType, StructType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala index 7b9a478b8be19..068303e25e2e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/TransformWithStateVariableUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate import org.json4s.DefaultFormats import org.json4s.JsonAST._ @@ -23,7 +23,8 @@ import org.json4s.jackson.JsonMethods import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.streaming.StateVariableType.StateVariableType +import org.apache.spark.sql.execution.streaming.operators.stateful.{SchemaValidationUtils, StatefulOperatorStateInfo} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType.StateVariableType import org.apache.spark.sql.execution.streaming.state.{OperatorInfoV1, OperatorStateMetadata, OperatorStateMetadataV2, StateStoreErrors, StateStoreId, StateStoreMetadataV2} import org.apache.spark.sql.streaming.{OutputMode, TimeMode} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala index 3cf3286fafb80..dfba0e1f12146 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImpl.scala @@ -14,7 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor import java.util import java.util.UUID @@ -26,13 +27,19 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.PRE_INIT -import org.apache.spark.sql.execution.streaming.StateVariableType._ -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils.{getExpirationMsRowSchema, getTTLRowKeySchema} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateStoreColumnFamilySchemaUtils, TimerKeyEncoder, TransformWithStateVariableInfo, TransformWithStateVariableUtils} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateVariableType._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils.{getExpirationMsRowSchema, getTTLRowKeySchema} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.StatefulProcessorHandleState.PRE_INIT +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables.{ListStateImpl, MapStateImpl, ValueStateImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{TimerStateImpl, TimerStateUtils} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.{ListStateImplWithTTL, MapStateImplWithTTL, TTLState, ValueStateImplWithTTL} +import org.apache.spark.sql.execution.streaming.runtime.{MicroBatchExecution, StreamExecution} import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, TimeMode, TTLConfig, ValueState} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +// scalastyle:on line.size.limit /** * Object used to assign/retrieve/remove grouping key passed implicitly for various state diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala index 64d87073ccf9f..4b593fe309472 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statefulprocessor/StatefulProcessorHandleImplBase.scala @@ -14,13 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.plans.logical.NoTime -import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.{INITIALIZED, PRE_INIT, StatefulProcessorHandleState, TIMER_PROCESSED} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.StatefulProcessorHandleState.{INITIALIZED, PRE_INIT, StatefulProcessorHandleState, TIMER_PROCESSED} import org.apache.spark.sql.execution.streaming.state.StateStoreErrors import org.apache.spark.sql.streaming.{StatefulProcessorHandle, TimeMode} +// scalastyle:on line.size.limit abstract class StatefulProcessorHandleImplBase( timeMode: TimeMode, keyExprEnc: ExpressionEncoder[Any]) extends StatefulProcessorHandle { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala index 32683aebd8c18..c6fe03da49ddf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateImpl.scala @@ -14,14 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors} import org.apache.spark.sql.streaming.ListState import org.apache.spark.sql.types.StructType +// scalastyle:on line.size.limit /** * Provides concrete implementation for list of values associated with a state variable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala index 66b6bd063ae6b..2275aba1a5887 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ListStateMetricsImpl.scala @@ -14,13 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.streaming.TransformWithStateVariableUtils.getRowCounterCFName +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateVariableUtils.getRowCounterCFName import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore} import org.apache.spark.sql.types._ +// scalastyle:on line.size.limit /** * Trait that provides helper methods to maintain metrics for a list state. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala index b71d625b118e1..007791ea55477 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/MapStateImpl.scala @@ -14,15 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.CompositeKeyStateEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils import org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, StateStore, StateStoreErrors, UnsafeRowPair} import org.apache.spark.sql.streaming.MapState import org.apache.spark.sql.types.StructType +// scalastyle:on line.size.limit /** * Class that provides a concrete implementation for map state associated with state diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala index d7e7a3d83f4df..2a4a79bca1e0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/statevariables/ValueStateImpl.scala @@ -14,13 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +// scalastyle:off line.size.limit +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore} import org.apache.spark.sql.streaming.ValueState +// scalastyle:on line.size.limit /** * Class that provides a concrete implementation for a single value state associated with state diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala index 984d650a27ccc..ec602988dc3e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/ExpiredTimerInfoImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers import org.apache.spark.sql.streaming.ExpiredTimerInfo diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala index 27c109f9de09b..647c0b3036a2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala @@ -14,13 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{EXPIRY_TIMESTAMP, KEY} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TimerKeyEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.streaming.TimeMode import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala index eb65aade67e36..234b80a84e8ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerValuesImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers import org.apache.spark.sql.streaming.TimerValues diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala index 4e32b80578155..08f97e38bd086 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ListStateImplWithTTL.scala @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors} import org.apache.spark.sql.streaming.{ListState, TTLConfig} import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala index aa4446af6da7b..f063354bc8c8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/MapStateImplWithTTL.scala @@ -14,12 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.CompositeKeyStateEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils import org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, StateStore, StateStoreErrors} import org.apache.spark.sql.streaming.{MapState, TTLConfig} import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala index 398088ab16978..548a47ea75e13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/TTLState.scala @@ -14,14 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl import java.time.Duration import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TTLEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.TWSMetricsUtils import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, RangeKeyScanStateEncoderSpec, StateStore} import org.apache.spark.sql.streaming.TTLConfig import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala index 836aac4cdb5a0..587da75993610 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/ttl/ValueStateImplWithTTL.scala @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._ +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateTypesEncoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateKeyValueRowSchemaUtils._ import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore} import org.apache.spark.sql.streaming.{TTLConfig, ValueState} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala index bf4333d8e9dd6..b15b93b47ada4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AcceptsLatestSeenOffsetHandler.scala @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.connector.read.streaming.{AcceptsLatestSeenOffset, SparkDataStream} +import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeq /** * This feeds "latest seen offset" to the sources that implement AcceptsLatestSeenOffset. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala index cb7e71bda84dd..43d5a50b2ccda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncLogPurge.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.concurrent.atomic.AtomicBoolean diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala index df3f1fdd8c903..2a87ba3380883 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncProgressTrackingMicroBatchExecution.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.concurrent._ import java.util.concurrent.atomic.AtomicLong @@ -24,6 +24,9 @@ import org.apache.spark.internal.LogKeys.{BATCH_ID, PRETTY_ID_STRING} import org.apache.spark.sql.catalyst.streaming.WriteToStream import org.apache.spark.sql.classic.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, OneTimeTrigger, ProcessingTimeTrigger} +import org.apache.spark.sql.execution.streaming.checkpointing.{AsyncCommitLog, AsyncOffsetSeqLog, CommitMetadata, OffsetSeq} +import org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreWriter import org.apache.spark.sql.streaming.Trigger import org.apache.spark.util.{Clock, ThreadUtils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala index e3a5ded83eb17..65113ce15c397 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AsyncStreamingQueryCheckpointMetadata.scala @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.concurrent.ThreadPoolExecutor import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.checkpointing.{AsyncCommitLog, AsyncOffsetSeqLog} import org.apache.spark.util.Clock /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala index f70c255c28edb..9a5cf89d03fab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowDataStreamWrapper.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{DELEGATE, READ_LIMIT} import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReadLimit, SparkDataStream, SupportsAdmissionControl, SupportsTriggerAvailableNow} import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.execution.streaming.Source /** * This class wraps a [[SparkDataStream]] and makes it support Trigger.AvailableNow, by overriding diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala index 3df358ae9bf80..ec63033edd40b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowMicroBatchStreamWrapper.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala index af5713144ebb2..bca4324447718 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/AvailableNowSourceWrapper.scala @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.streaming.{Offset, Source} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala index d76f3152286f2..8a90982b7c0c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/CompactibleFileStreamLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.io.{FileNotFoundException, InputStream, IOException, OutputStream} import java.nio.charset.StandardCharsets.UTF_8 @@ -27,9 +27,10 @@ import org.apache.hadoop.fs.Path import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization -import org.apache.spark.internal.{LogKeys} +import org.apache.spark.internal.LogKeys import org.apache.spark.sql.SparkSession import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala index 11bdfee460e66..d948325c9ae3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ContinuousRecordEndpoint.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import scala.collection.mutable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala index 0f25d0667a0ef..19867c5168373 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ErrorNotifier.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.concurrent.atomic.AtomicReference diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala index b259f9dbcdcb2..9ceee21a871c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamOptions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.Locale diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala index 801d42a1f6b2f..d5503f1c247da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSource.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.concurrent.ThreadPoolExecutor import java.util.concurrent.TimeUnit._ @@ -35,6 +35,8 @@ import org.apache.spark.sql.connector.read.streaming import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxBytes, ReadMaxFiles, SupportsAdmissionControl, SupportsTriggerAvailableNow} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} +import org.apache.spark.sql.execution.streaming.{Offset, Source} +import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala index 14653864a2922..451cf41152421 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceLog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.{LinkedHashMap => JLinkedHashMap} import java.util.Map.Entry @@ -23,7 +23,8 @@ import java.util.Map.Entry import scala.collection.mutable import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.streaming.FileStreamSource.FileEntry +import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog +import org.apache.spark.sql.execution.streaming.runtime.FileStreamSource.FileEntry import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala index ba79c77f38677..2cfece1942f0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/FileStreamSourceOffset.scala @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import scala.util.control.Exception._ import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization +import org.apache.spark.sql.execution.streaming.Offset + /** * Offset for the [[FileStreamSource]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala index f602de39ca9d0..938919ad40b64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID import java.util.concurrent.atomic.AtomicInteger @@ -38,6 +38,11 @@ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, MergingSessi import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataPartitionReader import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike import org.apache.spark.sql.execution.python.streaming.{FlatMapGroupsInPandasWithStateExec, TransformWithStateInPySparkExec} +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, OffsetSeqMetadata} +import org.apache.spark.sql.execution.streaming.operators.stateful.{SessionWindowStateStoreRestoreExec, SessionWindowStateStoreSaveExec, StatefulOperator, StatefulOperatorStateInfo, StateStoreRestoreExec, StateStoreSaveExec, StateStoreWriter, StreamingDeduplicateExec, StreamingDeduplicateWithinWatermarkExec, StreamingGlobalLimitExec, StreamingLocalLimitExec, UpdateEventTimeColumnExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExec +import org.apache.spark.sql.execution.streaming.operators.stateful.join.{StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.TransformWithStateExec import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSourceV1 import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadataReader, OperatorStateMetadataV1, OperatorStateMetadataV2, OperatorStateMetadataWriter, StateSchemaBroadcast, StateSchemaMetadata} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala index a27898cb0c9fc..19588082b3c54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/LongOffset.scala @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime + +import org.apache.spark.sql.execution.streaming.Offset /** * A simple offset for sources that produce a single linear stream of data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala index 16ca548fabb6b..32e2a0e53dd46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetadataLogFileIndex.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import scala.collection.mutable @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSink, FileStreamSinkLog} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala index 3919f8a2eb21f..0789ecee8533a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MetricsReporter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.time.ZonedDateTime import java.time.format.DateTimeFormatter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala index 7a9f5db8c5119..fc80cbae751f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/MicroBatchExecution.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import scala.collection.mutable.{Map => MutableMap} import scala.collection.mutable import scala.util.control.NonFatal -import org.apache.spark.internal.{LogKeys} +import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys.BATCH_ID import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp, FileSourceMetadataAttribute, LocalTimestamp} @@ -37,6 +37,10 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, StreamingDataSourceV2Relation, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} +import org.apache.spark.sql.execution.streaming.{AvailableNowTrigger, Offset, OneTimeTrigger, ProcessingTimeTrigger, Sink, Source} +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitMetadata, OffsetSeq, OffsetSeqMetadata} +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StatefulOpStateStoreCheckpointInfo, StateStoreWriter} +import org.apache.spark.sql.execution.streaming.runtime.AcceptsLatestSeenOffsetHandler import org.apache.spark.sql.execution.streaming.sources.{ForeachBatchSink, WriteToMicroBatchDataSource, WriteToMicroBatchDataSourceV1} import org.apache.spark.sql.execution.streaming.state.StateSchemaBroadcast import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala index a64ac31608cba..19aa068869dc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ProgressReporter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.time.Instant import java.time.ZoneId @@ -37,6 +37,8 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReportsSinkMetrics, ReportsSourceMetrics, SparkDataStream} import org.apache.spark.sql.execution.{QueryExecution, StreamSourceAwareSparkPlan} import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2ScanRelation, StreamWriterCommitProgress} +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata +import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StateStoreWriter} import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryIdleEvent, QueryProgressEvent} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala index 84f0961e4af12..2ec50b604ac8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/RateStreamOffset.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.json4s.DefaultFormats import org.json4s.jackson.Serialization diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala index a41abeb0a66b1..ee7bf67eb9121 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/ResolveWriteToStream.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement} import org.apache.spark.sql.connector.catalog.SupportsWrite import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala index 129cfed860eb6..e84ac4c1a1ed8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/SerializedOffset.scala @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime + +import org.apache.spark.sql.execution.streaming.Offset /** * Used when loading a JSON serialized offset from external storage. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala index 4fd124d1831fd..17095bcabf32a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecution.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.io.{InterruptedIOException, UncheckedIOException} import java.nio.channels.ClosedByInterruptException @@ -43,6 +43,8 @@ import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLi import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, SupportsTruncate, Write} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.StreamingExplainCommand +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitLog, OffsetSeqLog, OffsetSeqMetadata} +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperator, StateStoreWriter} import org.apache.spark.sql.execution.streaming.sources.{ForeachBatchUserFuncException, ForeachUserFuncException} import org.apache.spark.sql.execution.streaming.state.OperatorStateMetadataV2FileManager import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala index c5e14df3e20e1..b177bbdede740 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamExecutionContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID @@ -24,6 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.streaming.SparkDataStream +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata import org.apache.spark.util.Clock /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala index 19949d719e2ad..04e2dfb7833e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamMetadata.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.io.{InputStreamReader, OutputStreamWriter} import java.nio.charset.StandardCharsets @@ -29,7 +29,8 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala index 02f52bb30e1f3..a6fd103e8d6a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamProgress.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import scala.collection.immutable import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.execution.streaming.checkpointing.{OffsetSeq, OffsetSeqMetadata} /** * A helper class that looks like a Map[Source, Offset]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala index 27c76ba6bd67b..b92d862e17889 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingCheckpointConstants.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime object StreamingCheckpointConstants { val DIR_NAME_COMMITS = "commits" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala index 072ccb21e514a..cc7e92d9d7d1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryCheckpointMetadata.scala @@ -14,13 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitLog, OffsetSeqLog} /** * An interface for accessing the checkpoint metadata associated with a streaming query. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala index b3014ebc818da..d3762eada387e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryListenerBus.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala index c59b9584f5383..5585962ae16a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingQueryWrapper.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.UUID diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala index c5d5a79d34545..96f6340cb958d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/StreamingRelation.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -29,6 +29,7 @@ import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.{DataSource, FileFormat} +import org.apache.spark.sql.execution.streaming.Source import org.apache.spark.sql.sources.SupportsStreamSourceMetadataColumns object StreamingRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala index 5457387aa3f2e..8a175e633d49d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/TriggerExecutor.scala @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{ELAPSED_TIME, TRIGGER_INTERVAL} +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger import org.apache.spark.util.{Clock, SystemClock} trait TriggerExecutor { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala index aaf8cbd69ea20..b3d5baf0b5af0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkPropagator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.{util => jutil} @@ -24,6 +24,7 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeWatermarkExec, StateStoreWriter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala index 55eb823eadb68..6d94630d8c3bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/WatermarkTracker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util.{Locale, UUID} @@ -26,6 +26,7 @@ import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.streaming.operators.stateful.EventTimeWatermarkExec import org.apache.spark.sql.internal.SQLConf /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala index 48af1972e581c..68eb3cc7688d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/memory.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.runtime import java.util import java.util.concurrent.atomic.AtomicInteger @@ -37,6 +37,7 @@ import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, ReadLimit, SparkDataStream, SupportsTriggerAvailableNow} +import org.apache.spark.sql.execution.streaming.Offset import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala index 83c7ced1b6d80..978ec99ae9b39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSink.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.sinks import scala.util.control.NonFatal @@ -31,6 +31,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormat, FileFormatWriter} +import org.apache.spark.sql.execution.streaming.{ManifestFileCommitProtocol, Sink} +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala index 82fbc49de439c..71679e284f9d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sinks/FileStreamSinkLog.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.sinks import org.apache.hadoop.fs.FileStatus import org.apache.spark.internal.LogKeys._ import org.apache.spark.paths.SparkPath import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog import org.apache.spark.sql.internal.SQLConf /** 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 d0ba95ffd8a23..03884d02faeb7 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 @@ -32,7 +32,8 @@ 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.execution.streaming.{Offset => _, _} +import org.apache.spark.sql.execution.streaming.{Offset => _} +import org.apache.spark.sql.execution.streaming.runtime.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord, MemoryStreamBase} import org.apache.spark.util.RpcUtils /** 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 b166b92c3b898..80d3d97b12ed7 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 @@ -29,7 +29,8 @@ 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.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.checkpointing.HDFSMetadataLog +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{ManualClock, SystemClock, Utils} 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 ed93c6dda7a6c..243d6c784f5ae 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 @@ -31,7 +31,7 @@ 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.execution.streaming.runtime.LongOffset import org.apache.spark.unsafe.types.UTF8String /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 8f8ec84e2d10e..ac7f1a021960d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -34,8 +34,8 @@ import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.types.StructType import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala index ee3b3cdb6d9eb..c34545216fdaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala @@ -31,9 +31,11 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.v2.state.StateDataSourceErrors -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, CommitLog, MetadataVersionUtil, StateStoreWriter, StreamingQueryCheckpointMetadata} -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream -import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_OFFSETS +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, CommitLog, MetadataVersionUtil} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreWriter +import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_OFFSETS +import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryCheckpointMetadata import org.apache.spark.sql.execution.streaming.state.OperatorStateMetadataUtils.{OperatorStateMetadataReader, OperatorStateMetadataWriter} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index d3e3b52d9c24f..0063f960c4757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -41,7 +41,7 @@ import org.apache.spark.internal.{Logging, LogKeys, MessageWithContext} import org.apache.spark.internal.LogKeys.{DFS_FILE, VERSION_NUM} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index cf5f8ba5f2ebe..f49c79f96b9ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, AvroSerializer, import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StateStoreColumnFamilySchemaUtils} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.StateStoreColumnFamilySchemaUtils import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.{SCHEMA_ID_PREFIX_BYTES, STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 6bc3dd568af7b..78b8cc66f6db0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -32,7 +32,7 @@ import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager import org.apache.spark.sql.execution.streaming.state.StateStoreEncoding.Avro import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.Platform diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala index d67eb40fde2c2..7da8c5a6bd3ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream} import org.json4s.DefaultFormats import org.json4s.jackson.JsonMethods -import org.apache.spark.sql.execution.streaming.MetadataVersionUtil +import org.apache.spark.sql.execution.streaming.checkpointing.MetadataVersionUtil import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala index 143263075101c..ca18ce9067b36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala @@ -29,7 +29,8 @@ import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.{AvroDeserializer, AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.util.UnsafeRowUtils -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StatefulOperatorStateInfo} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo import org.apache.spark.sql.execution.streaming.state.SchemaHelper.{SchemaReader, SchemaWriter} import org.apache.spark.sql.execution.streaming.state.StateSchemaCompatibilityChecker.SCHEMA_FORMAT_V3 import org.apache.spark.sql.internal.{SessionState, SQLConf} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 2f3c05b72388a..604a27866f62e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -40,7 +40,8 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.UnsafeRowUtils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.execution.streaming.{StatefulOperatorStateInfo, StreamExecution} +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.execution.streaming.state.MaintenanceTaskType._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.{NextIterator, ThreadUtils, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index 9858b2494f84e..4c5dea63baeab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -30,8 +30,8 @@ import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.execution.streaming.CheckpointFileManager -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.execution.streaming.state.RecordType.RecordType import org.apache.spark.util.NextIterator import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala index 4026effcb088c..00bb7de46dc4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming.state -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SQLConf /** A class that contains configuration parameters for [[StateStore]]s. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala index a15412c3641da..d9b80ed99689d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala @@ -23,6 +23,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.classic.ClassicConversions.castToImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SessionState import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala index 76f64dcb64451..1c2abf5179f28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/StreamingQueryHistoryServerPlugin.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.ui import org.apache.spark.SparkConf import org.apache.spark.scheduler.SparkListener -import org.apache.spark.sql.execution.streaming.StreamingQueryListenerBus +import org.apache.spark.sql.execution.streaming.runtime.StreamingQueryListenerBus import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} import org.apache.spark.ui.SparkUI diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 9942918638c55..4b302c6a2108c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.{TableCapabilityCheck, V2SessionCatalog} -import org.apache.spark.sql.execution.streaming.ResolveWriteToStream +import org.apache.spark.sql.execution.streaming.runtime.ResolveWriteToStream import org.apache.spark.sql.expressions.UserDefinedAggregateFunction import org.apache.spark.sql.util.ExecutionListenerManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 635e183f9a025..af1f38caab686 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, PATH, VALUE} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.CacheManager -import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab, StreamingQueryStatusStore} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala index b92e361ef805b..17fb402457b74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.Optional -import org.apache.spark.sql.execution.streaming.GroupStateImpl -import org.apache.spark.sql.execution.streaming.GroupStateImpl._ +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._ /** * :: Experimental :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index c9c29ab4746b5..f31e5d2037ea1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.connector.expressions.{BucketTransform, ClusterByTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.FakeSourceOne import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index a15bbfe5c6574..653ad7bc34332 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -47,7 +47,7 @@ import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 1ed5ea4216a9f..e1a2fd33c7c93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 23481224887d4..9171e44571e88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelationWithTable} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.sources.SimpleScanSource diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 95301adb9b686..4da06ed82993d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.connector.catalog.{Table, TableCapability} 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} +import org.apache.spark.sql.execution.streaming.{Offset, Source} +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala index 0b313eb64f26a..ff6a215496bee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.{QueryExecution, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 8cf7d00ca2fe0..fcecaf25d4ca5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -46,7 +46,7 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProv import org.apache.spark.sql.execution.datasources.orc.OrcTest import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog -import org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.FileSystemBasedCheckpointFileManager import org.apache.spark.sql.execution.vectorized.ConstantColumnVector import org.apache.spark.sql.functions.{lit, lower, struct, sum, udf} import org.apache.spark.sql.internal.LegacyBehaviorPolicy.EXCEPTION diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala index 5ff459513e848..aac68e6389805 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.streaming.StreamTest class QueryPlanningTrackerEndToEndSuite extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/StateStoreBasicOperationsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/StateStoreBasicOperationsBenchmark.scala index 18d3b1f5da27f..ab03d9cb6a1f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/StateStoreBasicOperationsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/StateStoreBasicOperationsBenchmark.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StateStore, StateStoreConf, StateStoreId, StateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType, TimestampType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala index adcc771fe33ff..80877b0fb9f0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.noop import org.apache.spark.sql.classic.DataFrame -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.{StreamingQuery, StreamTest, Trigger} class NoopStreamSuite extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index be17d21339b7b..324fe148592a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.localDateTimeToMicros import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.SQLConf 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/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala index 8ee6d8762404c..be19981dc8a8c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.Assertions import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala index 36f21ddf8e09a..deda590645de5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, Row} import org.apache.spark.sql.catalyst.expressions.{BoundReference, GenericInternalRow} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil -import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog, StreamExecution} +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitLog, OffsetSeqLog} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala index 705d9f125964f..64d005c719b7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2.state import java.sql.Timestamp import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index 7d242c7444f13..1800319fb8b46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.{Encoders, Row} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBFileManager, RocksDBStateStoreProvider, TestClass} import org.apache.spark.sql.functions.{col, explode, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala index 5914abd11c01c..3b3e8687858dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/PythonStreamingDataSourceSuite.scala @@ -26,7 +26,9 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs} import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset} import org.apache.spark.sql.execution.python.PythonDataSourceSuiteBase -import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, OffsetSeqLog, ProcessingTimeTrigger} +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitLog, OffsetSeqLog} +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.StreamingQueryException import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServerSuite.scala index 76c9a1fbbd4e1..ff99b4ee280d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/streaming/TransformWithStateInPySparkStateServerSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{Encoder, Row} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.execution.streaming.{StatefulProcessorHandleImpl, StatefulProcessorHandleState} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{StatefulProcessorHandleImpl, StatefulProcessorHandleState} import org.apache.spark.sql.execution.streaming.state.StateMessage import org.apache.spark.sql.execution.streaming.state.StateMessage.{AppendList, AppendValue, Clear, ContainsKey, DeleteTimer, Exists, ExpiryTimerRequest, Get, GetProcessingTime, GetValue, GetWatermark, HandleState, Keys, ListStateCall, ListStateGet, ListStatePut, ListTimers, MapStateCall, ParseStringSchema, RegisterTimer, RemoveKey, SetHandleState, StateCallCommand, StatefulProcessorCall, TimerRequest, TimerStateCallCommand, TimerValueRequest, UpdateValue, UtilsRequest, Values, ValueStateCall, ValueStateUpdate} import org.apache.spark.sql.streaming.{ListState, MapState, TTLConfig, ValueState} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala index f9059ae1142f1..218b66b779463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala @@ -29,7 +29,9 @@ import org.scalatest.time.{Seconds, Span} import org.apache.spark.TestUtils import org.apache.spark.sql._ import org.apache.spark.sql.connector.read.streaming -import org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, ASYNC_PROGRESS_TRACKING_ENABLED, ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK} +import org.apache.spark.sql.execution.streaming.checkpointing.{AsyncCommitLog, AsyncOffsetSeqLog, OffsetSeq} +import org.apache.spark.sql.execution.streaming.runtime.{AsyncProgressTrackingMicroBatchExecution, MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming.runtime.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, ASYNC_PROGRESS_TRACKING_ENABLED, ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK} import org.apache.spark.sql.functions.{column, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryException, StreamTest, Trigger} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala index 8580aaf695231..788bd9ae33948 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala @@ -27,7 +27,8 @@ import org.apache.hadoop.fs._ import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, FileContextBasedCheckpointFileManager, FileSystemBasedCheckpointFileManager, HDFSMetadataLog} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala index 808ffe036f89d..5a1608cb6165a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala @@ -22,11 +22,12 @@ import java.nio.charset.StandardCharsets._ import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog import org.apache.spark.sql.test.SharedSparkSession class CompactibleFileStreamLogSuite extends SharedSparkSession { - import CompactibleFileStreamLog._ + import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog._ /** -- testing of `object CompactibleFileStreamLog` begins -- */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index 27cdeaeb46238..12f99a4c78d34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -28,13 +28,14 @@ import scala.util.Random import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path, RawLocalFileSystem} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSinkLog, SinkFileStatus} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.ArrayImplicits._ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSparkSession { - import CompactibleFileStreamLog._ + import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog._ import FileStreamSinkLog._ test("shouldRetain") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 08f245135f589..d6702c1e4ea50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -25,6 +25,7 @@ import scala.language.implicitConversions import org.scalatest.concurrent.Waiters._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.execution.streaming.checkpointing.{FileContextBasedCheckpointFileManager, FileSystemBasedCheckpointFileManager, HDFSMetadataLog} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.UninterruptibleThread diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala index 01b601783f3d4..4ec44eac22e36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{IntegerType, StructField, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIteratorSuite.scala index 163ff43a45aff..aa348d3df598e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MergingSortWithSessionWindowStateIteratorSuite.scala @@ -25,7 +25,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes -import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, PrefixKeyScanStateEncoderSpec, RocksDBStateStoreProvider, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId, StreamingSessionWindowStateManager} +import org.apache.spark.sql.execution.streaming.operators.stateful.{MergingSortWithSessionWindowStateIterator, StatefulOperatorStateInfo, StreamingSessionWindowStateManager} +import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, PrefixKeyScanStateEncoderSpec, RocksDBStateStoreProvider, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala index 1778b1fda7736..3fec6e816b839 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.connector.read.streaming import org.apache.spark.sql.connector.read.streaming.SparkDataStream +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, MicroBatchExecution, SerializedOffset, StreamExecution, StreamingExecutionRelation} import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest, Trigger} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala index 0859c2698e669..9c4a7b1879f61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.streaming import java.io.File import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.execution.streaming.checkpointing.{OffsetSeq, OffsetSeqLog, OffsetSeqMetadata} +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, SerializedOffset} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index bb7f8fc98d602..a45abbbd5542f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.streaming.runtime.ProcessingTimeExecutor import org.apache.spark.sql.streaming.util.StreamManualClock class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala index 87f8004ab9588..e1c5c1a3e0772 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala @@ -23,6 +23,7 @@ import java.util.UUID import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.sql.execution.streaming.runtime.StreamMetadata import org.apache.spark.sql.streaming.StreamTest class StreamMetadataSuite extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamRelationSuite.scala index e4ac17e0467ad..f7e2d069fd104 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamRelationSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{Project, SubqueryAlias} import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.runtime.StreamingRelation import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.IntegerType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/WatermarkTrackerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/WatermarkTrackerSuite.scala index 6018d286fc21e..052b77b5fa9b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/WatermarkTrackerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/WatermarkTrackerSuite.scala @@ -22,6 +22,8 @@ import java.util.UUID import scala.collection.mutable import org.apache.spark.sql.execution.{SparkPlan, UnionExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeStatsAccum, EventTimeWatermarkExec} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, WatermarkTracker} import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.streaming.StreamTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala index b5f23853fd5b2..f26635e7c67a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming.sources import java.io.ByteArrayOutputStream -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.{StreamTest, Trigger} class ConsoleWriteSupportSuite extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala index ec4aa8dacf008..de92aa6d998ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.ExecutorDeadException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.SerializeFromObjectExec -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.StateStoreCommitValidationFailed import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 32f92ce276a06..8c657c2e07160 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{ExecutorDeadException, SparkException} import org.apache.spark.sql.ForeachWriter -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession 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 aeb1bba31410d..f5e3cb6ce0aef 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 @@ -28,8 +28,8 @@ 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.StreamingDataSourceV2ScanRelation -import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, RateStreamOffset, StreamExecution} import org.apache.spark.sql.functions._ import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.util.CaseInsensitiveStringMap 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 2c17d75624d38..8b8ae57c82c4d 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 @@ -33,8 +33,8 @@ 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.StreamingDataSourceV2ScanRelation -import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.execution.streaming.runtime.{ContinuousRecordPartitionOffset, LongOffset, StreamExecution} 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/execution/streaming/state/FailureInjectionCheckpointFileManager.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala index 28e10affbbf2e..ec5d195f8f8ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala @@ -25,8 +25,8 @@ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.util.Progressable import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} -import org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} +import org.apache.spark.sql.execution.streaming.checkpointing.FileSystemBasedCheckpointFileManager /** * A wrapper file output stream that will throw exception in close() and put the underlying diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelperSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelperSuite.scala index 2456999b4382a..0369dd6c81ef5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelperSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelperSuite.scala @@ -22,7 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.execution.streaming.GroupStateImpl._ +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._ import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala index 476b43e42cb87..40caade2acb78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala @@ -23,7 +23,8 @@ import java.util.UUID import org.apache.spark.{SparkIllegalArgumentException, SparkUnsupportedOperationException} import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, ListStateImplWithTTL, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.ListStateImplWithTTL import org.apache.spark.sql.streaming.{ListState, TimeMode, TTLConfig, ValueState} /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala index 9a0a891d538ec..00855ba15f8df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala @@ -22,7 +22,8 @@ import java.util.UUID import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, MapStateImplWithTTL, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.MapStateImplWithTTL import org.apache.spark.sql.streaming.{ListState, MapState, TimeMode, TTLConfig, ValueState} import org.apache.spark.sql.types.{BinaryType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala index 2f3fd21bd3da0..f34ae3d32888b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala @@ -22,8 +22,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.execution.datasources.v2.state.{StateDataSourceUnspecifiedRequiredOption, StateSourceOptions} -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, LongOffset, MemoryStream, OffsetSeq, OffsetSeqLog} -import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_OFFSETS +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, OffsetSeq, OffsetSeqLog} +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream} +import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_OFFSETS import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, RunningCountStatefulProcessor, StreamTest, TimeMode} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala index cb80a77be39af..fee0dc2821fbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala @@ -24,7 +24,7 @@ import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala index 91117abf830e7..5892a93c32c4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreCheckpointFormatV2Suite.scala @@ -27,7 +27,8 @@ import org.scalatest.Tag import org.apache.spark.{SparkContext, SparkException, TaskContext} import org.apache.spark.sql.{DataFrame, ForeachWriter} import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.execution.streaming.{CommitLog, MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming.checkpointing.CommitLog +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef import org.apache.spark.sql.execution.streaming.state.StateStoreTestsHelper import org.apache.spark.sql.functions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala index d9de78dc04309..38e5b15465b89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters.SetHasAsScala import org.scalatest.time.{Millis, Minute, Seconds, Span} import org.apache.spark.memory.UnifiedMemoryManager -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions.{count, max} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreLockHardeningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreLockHardeningSuite.scala index 781dc32d0fa1a..cb59e005dee4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreLockHardeningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreLockHardeningSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkException, SparkFunSuite, SparkRuntimeException, TaskContext} import org.apache.spark.sql.catalyst.plans.PlanTestBase -import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.execution.streaming.state.StateStoreTestsHelper._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index e1f48441c4dbb..bb91878cd1d69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -37,7 +37,8 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.{StatefulOperatorStateInfo, StreamExecution} +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.internal.SQLConf 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/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 8aa3edafa8fc7..88c7598976505 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -41,8 +41,9 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, CreateAtomicTestManager, FileContextBasedCheckpointFileManager, FileSystemBasedCheckpointFileManager} -import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} +import org.apache.spark.sql.execution.streaming.CreateAtomicTestManager +import org.apache.spark.sql.execution.streaming.checkpointing.{CheckpointFileManager, FileContextBasedCheckpointFileManager, FileSystemBasedCheckpointFileManager} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala index 065124a784557..bbdc5440b39e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkUnsupportedOperationException -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo import org.apache.spark.sql.execution.streaming.state.StateStoreTestsHelper.newDir 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/execution/streaming/state/StateStoreCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCompatibilitySuite.scala index 4606c8fa792a7..2e896ce465402 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCompatibilitySuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala index 09118edc4357a..723bb0a876234 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -25,8 +25,9 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.classic.SparkSession -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions.{count, expr} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamingQuery, StreamTest, Trigger} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala index 4d842e1c87b8a..df4d19226b9d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreInstanceMetricSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.state import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters.MapHasAsScala -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.expr import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala index 7f2f3c49ec1ef..b4eeca748285d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.LocalSparkSession._ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.classic.ClassicConversions._ -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.util.{CompletionIterator, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 6e795f236c8b7..4d1e789a70b06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala index 0d74aade67194..a68965dd7845e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala @@ -22,7 +22,7 @@ import java.util.UUID import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StatefulProcessorHandleState} import org.apache.spark.sql.streaming.{TimeMode, TTLConfig} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManagerSuite.scala index 6685b140960d9..30ae505477ef4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManagerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.streaming.state import org.apache.spark.sql.catalyst.expressions.{Attribute, SpecificInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes +import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingAggregationStateManager import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{IntegerType, StructField, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManagerSuite.scala index 4d687a5655e0e..f3b76a8df0475 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StreamingSessionWindowStateManagerSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.{StatefulOperatorStateInfo, StreamingSessionWindowStateManager} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala index 781d0390f72cf..44364626c20dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala @@ -29,8 +29,9 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.LeftSide +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.join.{JoinStateManagerStoreGenerator, SymmetricHashJoinStateManager} +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.LeftSide import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala index 428845d5ebcbb..c74599a6aa746 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/TimerSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution.streaming.state import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, TimerStateImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.TimerStateImpl import org.apache.spark.sql.streaming.TimeMode /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index d35d0c6b405b4..fbe33ddc32db2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -29,7 +29,9 @@ import org.apache.spark.{SparkException, SparkUnsupportedOperationException, Tas import org.apache.spark.TaskContext.withTaskContext import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} -import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl, StreamExecution, ValueStateImplWithTTL} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.{ImplicitGroupingKeyTracker, StatefulProcessorHandleImpl} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.ValueStateImplWithTTL +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{TimeMode, TTLConfig, ValueState} import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/AcceptsLatestSeenOffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/AcceptsLatestSeenOffsetSuite.scala index 2dadbbe7cb237..2a4abd99f6c19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/AcceptsLatestSeenOffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/AcceptsLatestSeenOffsetSuite.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.connector.read.streaming import org.apache.spark.sql.connector.read.streaming.{AcceptsLatestSeenOffset, SparkDataStream} -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.{Offset, Source} +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.execution.streaming.sources.{ContinuousMemoryStream, ContinuousMemoryStreamOffset} import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala index 1af35d0a1db4d..aa5826572240f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/CommitLogSuite.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayInputStream, FileInputStream, FileOutputStream} import java.nio.file.Path import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.streaming.{CommitLog, CommitMetadata} +import org.apache.spark.sql.execution.streaming.checkpointing.{CommitLog, CommitMetadata} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala index 7777887ec62c9..3bc7df41a147d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.streaming import org.scalatest.Assertions -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.state.StreamingAggregationStateManager +import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingAggregationStateManager +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 04ddd79d028cb..db28ef4fc35a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -32,7 +32,8 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.operators.stateful.{EventTimeStats, StateStoreSaveExec} +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.functions.{count, expr, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 9742a00454569..067b0ca285d54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -38,7 +38,9 @@ import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.ManifestFileCommitProtocol +import org.apache.spark.sql.execution.streaming.runtime._ +import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSink, FileStreamSinkLog, SinkFileStatus} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index a6a044c302ce3..15a0f048dd8a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -39,7 +39,9 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.connector.read.streaming.ReadLimit import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap, SourceFileArchiver} +import org.apache.spark.sql.execution.streaming.runtime.{CleanSourceMode, FileStreamOptions, FileStreamSource, FileStreamSourceLog, FileStreamSourceOffset, MemoryStream, SerializedOffset, StreamExecution, StreamingExecutionRelation, StreamingQueryWrapper, StreamingRelation} +import org.apache.spark.sql.execution.streaming.runtime.FileStreamSource.{FileEntry, SeenFilesMap, SourceFileArchiver} +import org.apache.spark.sql.execution.streaming.sinks.{FileStreamSink, FileStreamSinkLog, SinkFileStatus} import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -1531,7 +1533,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { batchId: Long, expectedBatches: Int, expectedCompactInterval: Int): Boolean = { - import CompactibleFileStreamLog._ + import org.apache.spark.sql.execution.streaming.runtime.CompactibleFileStreamLog._ val fileSource = getSourcesFromStreamingQuery(execution).head val metadataLog = fileSource invokePrivate _metadataLog() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateDistributionSuite.scala index 3320a682b1240..de81a3127e41a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateDistributionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateDistributionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql.IntegratedUDFTestUtils.{shouldTestPandasUDFs, TestGroupedMapPandasUDFWithState} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update import org.apache.spark.sql.execution.python.streaming.FlatMapGroupsInPandasWithStateExec -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.util.{StatefulOpClusteredDistributionTestHelper, StreamManualClock} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala index 9c5714f682603..0017531e41e1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsInPandasWithStateSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.plans.logical.{NoTimeout, ProcessingTimeTimeout} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Complete, Update} import org.apache.spark.sql.execution.python.streaming.FlatMapGroupsInPandasWithStateExec -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.{lit, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala index 942cfe3717943..27c78e5865b02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.streaming import java.io.File import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update -import org.apache.spark.sql.execution.streaming.{FlatMapGroupsWithStateExec, MemoryStream} +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExec +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.GroupStateTimeout.ProcessingTimeTimeout import org.apache.spark.sql.streaming.util.{StatefulOpClusteredDistributionTestHelper, StreamManualClock} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index d99e72f914908..93efbe3b3cf5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -32,8 +32,9 @@ import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, MemoryStateStore, RocksDBStateStoreProvider, StateStore} +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.{FlatMapGroupsWithStateExec, FlatMapGroupsWithStateExecHelper, FlatMapGroupsWithStateUserFuncException} +import org.apache.spark.sql.execution.streaming.runtime._ +import org.apache.spark.sql.execution.streaming.state.{MemoryStateStore, RocksDBStateStoreProvider, StateStore} import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -52,7 +53,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { import testImplicits._ import FlatMapGroupsWithStateSuite._ - import GroupStateImpl._ + import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl._ import GroupStateTimeout._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala index 418fbd5603a80..93d6e469c5ccb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Dataset, KeyValueGroupedDataset} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update -import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.FlatMapGroupsWithStateExecHelper +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite.{assertCanGetProcessingTime, assertCannotGetWatermark} import org.apache.spark.sql.streaming.GroupStateTimeout.{EventTimeTimeout, NoTimeout, ProcessingTimeTimeout} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala index 69362dd60d889..7df9e1f335ae4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/GroupStateSuite.scala @@ -21,8 +21,8 @@ import java.sql.Date import org.apache.spark.{SparkFunSuite, SparkUnsupportedOperationException} import org.apache.spark.api.java.Optional -import org.apache.spark.sql.execution.streaming.GroupStateImpl -import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.flatmapgroupswithstate.GroupStateImpl.NO_TIMESTAMP import org.apache.spark.sql.streaming.GroupStateTimeout.{EventTimeTimeout, NoTimeout, ProcessingTimeTimeout} class GroupStateSuite extends SparkFunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala index 65ac6712ab45f..4a4decaadfa92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.tags.SlowSQLTest @SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala index 980f9f48dcb05..21bf370f82a5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MultiStatefulOperatorsSuite.scala @@ -22,7 +22,9 @@ import java.sql.Timestamp import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.execution.streaming.{MemoryStream, StateStoreSaveExec, StreamingSymmetricHashJoinExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.StateStoreSaveExec +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinExec +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala index f208f9bd9b6e3..8680cefb7caae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, SerializedOffset} +import org.apache.spark.sql.execution.streaming.Offset +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, SerializedOffset} trait OffsetSuite extends SparkFunSuite { /** Creates test to check all the comparisons of offsets given a `one` that is less than `two`. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala index c417693b5d7a6..7dd2e063c9c51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ReportSinkMetricsSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapabi import org.apache.spark.sql.connector.read.streaming.ReportsSinkMetrics import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.sources.PackedRowWriterFactory import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdateAsAppend} import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala index 5f9ff25e16e73..e9bfaf2fc56a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.util.ArrayImplicits._ trait StateStoreMetricsTest extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index f9bb9812b8cf9..cbb2eba7ecc89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -42,6 +42,8 @@ import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingLocalLimitExec +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, StreamExecution, StreamingExecutionRelation, StreamingQueryWrapper, StreamingRelation} import org.apache.spark.sql.execution.streaming.sources.{ContinuousMemoryStream, ForeachBatchUserFuncException, MemorySink} import org.apache.spark.sql.execution.streaming.state.{KeyStateEncoderSpec, StateSchemaProvider, StateStore, StateStoreConf, StateStoreId, StateStoreProvider} import org.apache.spark.sql.expressions.Window 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 a6efc2d8fa9c0..5b084c89eba1a 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 @@ -40,8 +40,9 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2ScanRelation -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.operators.stateful.StatefulOperator +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, MemoryStreamBase, MemoryStreamTable, MicroBatchExecution, StreamExecution, StreamingExecutionRelation, StreamingQueryWrapper} import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.streaming.StreamingQueryListener._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala index da2525606a2e2..465da3cd469f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala @@ -23,7 +23,8 @@ import org.scalatest.Assertions import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.execution.aggregate.BaseAggregateExec -import org.apache.spark.sql.execution.streaming.{MemoryStream, StateStoreRestoreExec, StateStoreSaveExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.{StateStoreRestoreExec, StateStoreSaveExec} +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode.Update diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index ca66018b51bf0..7825730d901da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -35,8 +35,10 @@ import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.operators.stateful.{StateStoreRestoreExec, StateStoreSaveExec, StreamingAggregationStateManager} +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.sources.MemorySink -import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreValueSchemaNotCompatible, StreamingAggregationStateManager} +import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreValueSchemaNotCompatible} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala index e3fd220d2d6cf..449ba9aaa18cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.streaming import java.io.File import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingDeduplicateExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.StreamingDeduplicateExec +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StatefulOpClusteredDistributionTestHelper import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 89fb80b9fd2f8..832b22d6304fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StringType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala index af86e6ec88996..a6223cef32da2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql.{AnalysisException, Dataset, SaveMode} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 47f9a8f80e23f..22028a585e229 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -35,7 +35,10 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.operators.stateful.join.{StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf 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 8a74355ed9305..3cc2805a4dd1d 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 @@ -31,7 +31,7 @@ import org.apache.spark.SparkException import org.apache.spark.scheduler._ import org.apache.spark.sql.{Encoder, Row, SparkSession} import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit} -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.streaming.ui.StreamingQueryStatusListener diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 305d9ca8f451d..2353b6a4e9252 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS import org.apache.spark.sql.streaming.StreamingQueryListener._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index 53cbbe6e786fe..fea218500677c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException import org.apache.spark.sql.{Dataset, Encoders} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2ScanRelation -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.BlockingSource import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala index f651bfb7f3c72..1c702faf40c38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryOptimizationCorrectnessSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import java.sql.Timestamp import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.{count, expr, lit, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index e748ae8e7d7df..958b8b47c759a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite._ 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 2a7c39d8bb00f..df5d49e03ee4a 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 @@ -45,6 +45,8 @@ import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit} import org.apache.spark.sql.execution.exchange.{REQUIRED_BY_STATEFUL_OPERATOR, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.checkpointing.OffsetSeqMetadata +import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, MetricsReporter, StreamExecution, StreamingExecutionRelation, StreamingQueryWrapper} import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala index 44c909d8ae2d7..069fc717deabf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.sql.SaveMode import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.streaming.test.{InMemoryStreamTable, InMemoryStreamTableCatalog} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{LongType, StructField, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala index 174eebd519304..e4221f2451790 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala @@ -22,7 +22,8 @@ import java.io.File import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.execution.aggregate.BaseAggregateExec -import org.apache.spark.sql.execution.streaming.{MemoryStream, SessionWindowStateStoreRestoreExec, SessionWindowStateStoreSaveExec} +import org.apache.spark.sql.execution.streaming.operators.stateful.{SessionWindowStateStoreRestoreExec, SessionWindowStateStoreSaveExec} +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.functions.{count, session_window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StatefulOpClusteredDistributionTestHelper diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala index ec3c145af686c..78e2c192ed50f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoder, Encoders} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{HDFSBackedStateStoreProvider, RocksDBStateStoreProvider} import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingStateStoreFormatCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingStateStoreFormatCompatibilitySuite.scala index 2c4a98ac4af8e..e14d0adc707c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingStateStoreFormatCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingStateStoreFormatCompatibilitySuite.scala @@ -24,7 +24,7 @@ import scala.annotation.tailrec import org.apache.spark.{SparkException, SparkUnsupportedOperationException} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{StateStoreKeyRowFormatValidationFailure, StateStoreValueRowFormatValidationFailure} import org.apache.spark.sql.functions._ import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSymmetricHashJoinHelperSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSymmetricHashJoinHelperSuite.scala index b38d5d7dbce9d..3336cc680ac0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSymmetricHashJoinHelperSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSymmetricHashJoinHelperSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal} import org.apache.spark.sql.execution.LocalTableScanExec -import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.JoinConditionSplitPredicates +import org.apache.spark.sql.execution.streaming.operators.stateful.join.StreamingSymmetricHashJoinHelper.JoinConditionSplitPredicates import org.apache.spark.sql.types._ class StreamingSymmetricHashJoinHelperSuite extends StreamTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala index b8932a9626098..b0a76e170ff58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala index 802535a3bb375..15312883d4d61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.streaming import java.time.Duration import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, MemoryStream} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.ListStateImplWithTTL +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala index 70ab969975df2..fd73517d81816 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala index a94deaf23b2c5..9470d2b65b375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.streaming import java.time.Duration import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{MapStateImplWithTTL, MemoryStream} +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.MapStateImplWithTTL +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateAvroSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateAvroSuite.scala index 8da9edda02ad6..f0583e5e6830b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateAvroSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateAvroSuite.scala @@ -27,8 +27,9 @@ import org.scalatest.time.{Seconds, Span} import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, MemoryStream, MicroBatchExecution} -import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_OFFSETS +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, MicroBatchExecution} +import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_OFFSETS import org.apache.spark.sql.execution.streaming.state.{OperatorStateMetadataV2, RocksDBStateStoreProvider, StateStoreInvalidValueSchemaEvolution, StateStoreValueSchemaEvolutionThresholdExceeded} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala index 553e28d4c0e14..7d065d561feb8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateChainingSuite.scala @@ -23,7 +23,7 @@ import java.time.{Instant, LocalDateTime, ZoneId} import org.apache.spark.{SparkRuntimeException, SparkThrowable} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.ExtendedAnalysisException -import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamExecution} import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.functions.window import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala index 3f0bb060d36de..f6f3b2bd8b795 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateClusterSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.sql.{Dataset, Encoders, Row, SparkSession} import org.apache.spark.sql.LocalSparkSession.withSparkSession -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider import org.apache.spark.sql.internal.SQLConf import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index e7435fb02aad7..5f4de279724a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql.{DataFrame, Dataset, Encoders, KeyValueGroupedDataset} import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.functions.{col, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 6127bf5395f35..2a1ec4c7ab611 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -32,8 +32,10 @@ import org.apache.spark.sql.{Dataset, Encoders, Row} import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.DIR_NAME_OFFSETS +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{TransformWithStateExec, TransformWithStateOperatorProperties, TransformWithStateVariableUtils} +import org.apache.spark.sql.execution.streaming.runtime._ +import org.apache.spark.sql.execution.streaming.runtime.StreamingCheckpointConstants.DIR_NAME_OFFSETS import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala index a499ca3f25b00..4fb1131440f79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import java.sql.Timestamp import java.time.Duration -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithEncodingTypes, AlsoTestWithRocksDBFeatures, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateUnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateUnsafeRowSuite.scala index 1988a458ed1c8..f578a209b18d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateUnsafeRowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateUnsafeRowSuite.scala @@ -21,7 +21,7 @@ import org.scalactic.source.Position import org.scalatest.Tag import org.apache.spark.SparkUnsupportedOperationException -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state.{RocksDBStateStoreProvider, StateStoreValueSchemaNotCompatible} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala index b01cb67a6b786..a42bf4ac97aea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithValueStateTTLSuite.scala @@ -23,7 +23,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.Encoders -import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, MemoryStream, ValueStateImpl, ValueStateImplWithTTL} +import org.apache.spark.sql.execution.streaming.checkpointing.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statevariables.ValueStateImpl +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.ttl.ValueStateImplWithTTL +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala index 659e2198f4377..3741ee8ab1feb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TriggerAvailableNowSuite.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connector.read.streaming import org.apache.spark.sql.connector.read.streaming.{ReadLimit, SupportsAdmissionControl} -import org.apache.spark.sql.execution.streaming.{LongOffset, MemoryStream, MicroBatchExecution, MultiBatchExecutor, Offset, SerializedOffset, SingleBatchExecutor, Source, StreamingExecutionRelation, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.{Offset, Source} + import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, MicroBatchExecution, MultiBatchExecutor, SerializedOffset, SingleBatchExecutor, StreamingExecutionRelation, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala index d7aa99c30aad6..068001b06d903 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming.continuous -import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.runtime.StreamExecution import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.streaming.Trigger import org.apache.spark.tags.SlowSQLTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index f1a63ac0f7857..c70f21ae144b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -22,8 +22,8 @@ import java.sql.Timestamp import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} import org.apache.spark.sql._ -import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf.{CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE, MIN_BATCHES_TO_RETAIN} 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 669fedcbe4b78..ab49fc669ed5d 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 @@ -29,7 +29,9 @@ import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderF import org.apache.spark.sql.connector.write.{LogicalWriteInfo, PhysicalWriteInfo, 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.execution.streaming.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.Sink +import org.apache.spark.sql.execution.streaming.runtime.{RateStreamOffset, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 6ea610857b077..74db2a3843d76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.classic.Dataset.ofRows import org.apache.spark.sql.execution.datasources.DataSourceUtils -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.{Offset, Sink, Source} +import org.apache.spark.sql.execution.streaming.runtime._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamingQueryException, StreamTest} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index 1c1575b06e5ce..89f6556229527 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.connector.{FakeV2Provider, FakeV2ProviderWithCustomS import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog, MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability, TableInfo, V2TableWithV1Fallback} import org.apache.spark.sql.connector.expressions.{ClusterByTransform, FieldReference, Transform} import org.apache.spark.sql.connector.read.ScanBuilder -import org.apache.spark.sql.execution.streaming.{MemoryStream, MemoryStreamScanBuilder, StreamingQueryWrapper} +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, MemoryStreamScanBuilder, StreamingQueryWrapper} import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala index 6550316bcc47c..ef79ddb645174 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, SupportsRe import org.apache.spark.sql.connector.catalog.TableCapability.CONTINUOUS_READ import org.apache.spark.sql.connector.read.{streaming, InputPartition, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, PartitionOffset} -import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Source} +import org.apache.spark.sql.execution.streaming.{Offset, Source} +import org.apache.spark.sql.execution.streaming.runtime.LongOffset import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.types.{LongType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala index 6865e5e0269b9..b234adc76d557 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.streaming.util import java.util.concurrent.CountDownLatch import org.apache.spark.sql.{DataFrame, SQLContext} -import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Sink, Source} +import org.apache.spark.sql.execution.streaming.{Offset, Sink, Source} +import org.apache.spark.sql.execution.streaming.runtime.LongOffset import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{IntegerType, StructField, StructType} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 89d34a3fa1977..f9bebce7cbfab 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.command.CommandCheck import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.TableCapabilityCheck -import org.apache.spark.sql.execution.streaming.ResolveWriteToStream +import org.apache.spark.sql.execution.streaming.runtime.ResolveWriteToStream import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.execution.PruneHiveTablePartitions diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index e43d1cdb19d4d..d3f625542d96f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2650,7 +2650,7 @@ class HiveDDLSuite } test("SPARK-21216: join with a streaming DataFrame") { - import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import testImplicits._ implicit val _sqlContext = spark.sqlContext diff --git a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala index 6b22253c4ae09..b58f6e2297fb6 100644 --- a/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala +++ b/sql/pipelines/src/main/scala/org/apache/spark/sql/pipelines/graph/elements.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.classic.{DataFrame, SparkSession} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.common.DatasetType import org.apache.spark.sql.pipelines.util.{ BatchReadOptions, diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala index 7ef3ba21d70ab..f37716b4a24d3 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.pipelines.graph import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.utils.{PipelineTest, TestGraphRegistrationContext} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala index 8f4780e9ea5dd..2c0e2a728c69f 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.pipelines.graph import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.Union -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.utils.{PipelineTest, TestGraphRegistrationContext} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala index 818fb3af176a8..72e292ec5070a 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/MaterializeTablesSuite.scala @@ -22,7 +22,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SparkThrowable import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.Expressions -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.graph.DatasetManager.TableMaterializationException import org.apache.spark.sql.pipelines.utils.{BaseCoreExecutionTest, TestGraphRegistrationContext} import org.apache.spark.sql.test.SharedSparkSession diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala index 75ef78cecda22..4aaa139378b93 100644 --- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala +++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/TriggeredGraphExecutionSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.{functions, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, TableCatalog} -import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream import org.apache.spark.sql.pipelines.common.{FlowStatus, RunState} import org.apache.spark.sql.pipelines.graph.TriggeredGraphExecution.StreamState import org.apache.spark.sql.pipelines.logging.EventLevel