├── .github ├── FUNDING.yml └── workflows │ └── deploying-to-github-pages.yml ├── .gitignore ├── LICENSE ├── README.md ├── docs ├── AcceptsLatestSeenOffset.md ├── AcceptsLatestSeenOffsetHandler.md ├── AvailableNowDataStreamWrapper.md ├── CheckpointFileManager.md ├── CommitLog.md ├── CommitMetadata.md ├── ContinuousDataSourceRDD.md ├── ContinuousQueuedDataReader.md ├── ContinuousStream.md ├── ContinuousWriteRDD.md ├── DataReaderThread.md ├── DataSource.md ├── DataStreamReader.md ├── DataStreamWriter.md ├── EpochCoordinator.md ├── EpochCoordinatorRef.md ├── EpochMarkerGenerator.md ├── EpochTracker.md ├── FileContextBasedCheckpointFileManager.md ├── FileSystemBasedCheckpointFileManager.md ├── GroupState.md ├── GroupStateImpl.md ├── GroupStateTimeout.md ├── HDFSMetadataLog.md ├── IncrementalExecution.md ├── MetadataLog.md ├── MicroBatchStream.md ├── Offset.md ├── OffsetSeq.md ├── OffsetSeqLog.md ├── OffsetSeqMetadata.md ├── OutputMode.md ├── PartitionOffset.md ├── ProcessingTimeExecutor.md ├── ProgressReporter.md ├── ReadLimit.md ├── ReportsSourceMetrics.md ├── SQLConf.md ├── Sink.md ├── Source.md ├── SparkDataStream.md ├── StreamExecution.md ├── StreamMetadata.md ├── StreamProgress.md ├── StreamSinkProvider.md ├── StreamSourceProvider.md ├── StreamingDataWriterFactory.md ├── StreamingQuery.md ├── StreamingQueryListenerBus.md ├── StreamingQueryManager.md ├── StreamingQueryWrapper.md ├── StreamingWrite.md ├── SupportsAdmissionControl.md ├── SupportsTriggerAvailableNow.md ├── Trigger.md ├── TriggerExecutor.md ├── UnsupportedOperationChecker.md ├── arbitrary-stateful-streaming-aggregation │ ├── FlatMapGroupsWithStateExecHelper.md │ ├── InputProcessor.md │ ├── StateManager.md │ ├── StateManagerImplBase.md │ ├── StateManagerImplV1.md │ ├── StateManagerImplV2.md │ └── index.md ├── batch-processing-time.md ├── configuration-properties.md ├── continuous-execution │ ├── ContinuousExecution.md │ ├── ContinuousReadSupport.md │ ├── ContinuousReader.md │ └── index.md ├── datasources │ ├── ForeachBatchSink.md │ ├── console │ │ ├── ConsoleSinkProvider.md │ │ ├── ConsoleWriter.md │ │ └── index.md │ ├── file │ │ ├── CompactibleFileStreamLog.md │ │ ├── FileStreamSink.md │ │ ├── FileStreamSinkLog.md │ │ ├── FileStreamSource.md │ │ ├── FileStreamSourceCleaner.md │ │ ├── FileStreamSourceLog.md │ │ ├── ManifestFileCommitProtocol.md │ │ ├── MetadataLogFileIndex.md │ │ ├── SinkFileStatus.md │ │ └── index.md │ ├── foreach │ │ ├── ForeachSink.md │ │ ├── ForeachWriter.md │ │ ├── ForeachWriterProvider.md │ │ ├── ForeachWriterTable.md │ │ └── index.md │ ├── index.md │ ├── memory │ │ ├── ContinuousMemoryStream.md │ │ ├── MemoryPlan.md │ │ ├── MemorySink.md │ │ ├── MemoryStream.md │ │ ├── MemoryStreamBase.md │ │ ├── MemoryStreamingWrite.md │ │ └── index.md │ ├── rate-micro-batch │ │ ├── RatePerMicroBatchProvider.md │ │ ├── RatePerMicroBatchStream.md │ │ ├── RatePerMicroBatchTable.md │ │ ├── index.md │ │ └── options.md │ ├── rate │ │ ├── RateStreamContinuousReader.md │ │ ├── RateStreamProvider.md │ │ └── index.md │ └── socket │ │ ├── TextSocketSource.md │ │ ├── TextSocketSourceProvider.md │ │ └── index.md ├── demo │ ├── StateStoreSaveExec-Complete.md │ ├── StateStoreSaveExec-Update.md │ ├── StreamingQueryManager-awaitAnyTermination-resetTerminated.md │ ├── arbitrary-stateful-streaming-aggregation-flatMapGroupsWithState.md │ ├── current_timestamp.md │ ├── custom-sink-webui.md │ ├── deep-dive-into-filestreamsink.md │ ├── exploring-checkpointed-state.md │ ├── groupBy-running-count-complete.md │ ├── groupByKey-count-Update.md │ ├── index.md │ ├── kafka-data-source.md │ ├── rocksdb-state-store-for-streaming-aggregation.md │ ├── spark-sql-streaming-demo-FlatMapGroupsWithStateExec.md │ ├── stream-stream-inner-join.md │ ├── streaming-watermark.md │ ├── streaming-windowed-aggregation.md │ ├── using-file-streaming-source.md │ └── watermark-aggregation-append.md ├── execution-planning-strategies │ ├── FlatMapGroupsWithStateStrategy.md │ ├── StatefulAggregationStrategy.md │ ├── StreamingDeduplicationStrategy.md │ ├── StreamingGlobalLimitStrategy.md │ ├── StreamingJoinStrategy.md │ ├── StreamingRelationStrategy.md │ └── index.md ├── features │ └── index.md ├── images │ ├── ContinuousExecution-addOffset.png │ ├── DataStreamReader-SparkSession-StreamingRelation.png │ ├── FlatMapGroupsWithStateExec-webui-query-details.png │ ├── KafkaOffsetReader-fetchSpecificOffsets.png │ ├── KafkaSource-creating-instance.png │ ├── KafkaSource-initialPartitionOffsets.png │ ├── KeyToNumValuesStore-KeyWithIndexToValueStore.png │ ├── MicroBatchExecution-constructNextBatch.png │ ├── MicroBatchExecution.png │ ├── OneSideHashJoiner.png │ ├── ProgressReporter-currentDurationsMs.png │ ├── ProgressReporter-finishTrigger-timestamps.png │ ├── ProgressReporter-updateProgress.png │ ├── StateStoreRDD-SparkPlans-LogicalPlans-operators.png │ ├── StateStoreRDD-StateStoreCoordinator.png │ ├── StateStoreRestoreExec-IncrementalExecution.png │ ├── StateStoreRestoreExec-StatefulAggregationStrategy.png │ ├── StateStoreRestoreExec-webui-query-details.png │ ├── StateStoreSaveExec-IncrementalExecution.png │ ├── StateStoreSaveExec-StateStoreRDD-count.png │ ├── StateStoreSaveExec-StateStoreRDD.png │ ├── StateStoreSaveExec-StatefulAggregationStrategy.png │ ├── StateStoreSaveExec-webui-query-details.png │ ├── StreamExecution-creating-instance.png │ ├── StreamExecution-getBatchDescriptionString-webUI.png │ ├── StreamExecution-runBatch-addBatch.png │ ├── StreamExecution-runBatch-getBatch.png │ ├── StreamExecution-runBatch-newBatchesPlan.png │ ├── StreamExecution-runBatch-nextBatch.png │ ├── StreamExecution-runBatch-queryPlanning.png │ ├── StreamExecution-runBatches.png │ ├── StreamExecution-start.png │ ├── StreamExecution-uniqueSources.png │ ├── StreamingDeduplicateExec-StreamingDeduplicationStrategy.png │ ├── StreamingDeduplicateExec-webui-query-details.png │ ├── StreamingExecutionRelation.png │ ├── StreamingQueryListener-onQueryProgress.png │ ├── StreamingQueryListener-onQueryStarted.png │ ├── StreamingQueryListener-onQueryTerminated.png │ ├── StreamingQueryListenerBus.png │ ├── StreamingQueryManager-createQuery.png │ ├── StreamingQueryManager-notifyQueryTermination.png │ ├── StreamingQueryManager-postListenerEvent.png │ ├── StreamingQueryManager.png │ ├── StreamingRelation.png │ ├── StreamingRelationStrategy-apply.png │ ├── StreamingSymmetricHashJoinExec-webui-query-details.png │ ├── SymmetricHashJoinStateManager.png │ ├── memory-sink-webui-createviewcommand.png │ ├── webui-completed-streaming-queries.png │ ├── webui-spark-job-streaming-query-started.png │ ├── webui-sql-completed-queries-one-per-batch.png │ ├── webui-sql-completed-queries-three-per-batch.png │ ├── webui-statistics-aggregated-number-of-rows-dropped-by-watermark.png │ ├── webui-statistics-aggregated-number-of-total-state-rows.png │ ├── webui-statistics-aggregated-number-of-updated-state-rows.png │ ├── webui-statistics-aggregated-state-memory-used-in-bytes.png │ ├── webui-statistics-basic-info.png │ ├── webui-statistics-global-watermark-gap.png │ └── workshop-tweet.png ├── index.md ├── internals.md ├── join │ ├── .pages │ ├── JoinStateWatermarkPredicate.md │ ├── JoinStateWatermarkPredicates.md │ ├── KeyToNumValuesStore.md │ ├── KeyWithIndexToValueStore.md │ ├── OneSideHashJoiner.md │ ├── StateStoreAwareZipPartitionsHelper.md │ ├── StateStoreAwareZipPartitionsRDD.md │ ├── StateStoreHandler.md │ ├── StreamingJoinHelper.md │ ├── StreamingSymmetricHashJoinHelper.md │ ├── SymmetricHashJoinStateManager.md │ └── index.md ├── kafka │ ├── CachedKafkaConsumer.md │ ├── ConsumerStrategy.md │ ├── KafkaBatch.md │ ├── KafkaBatchInputPartition.md │ ├── KafkaContinuousInputPartition.md │ ├── KafkaContinuousStream.md │ ├── KafkaDataConsumer.md │ ├── KafkaMicroBatchInputPartition.md │ ├── KafkaMicroBatchInputPartitionReader.md │ ├── KafkaMicroBatchStream.md │ ├── KafkaOffsetRangeCalculator.md │ ├── KafkaOffsetRangeLimit.md │ ├── KafkaOffsetReader.md │ ├── KafkaOffsetReaderAdmin.md │ ├── KafkaOffsetReaderConsumer.md │ ├── KafkaRelation.md │ ├── KafkaScan.md │ ├── KafkaSink.md │ ├── KafkaSource.md │ ├── KafkaSourceInitialOffsetWriter.md │ ├── KafkaSourceOffset.md │ ├── KafkaSourceProvider.md │ ├── KafkaSourceRDD.md │ ├── KafkaStreamWriterFactory.md │ ├── KafkaStreamingWrite.md │ ├── KafkaTable.md │ ├── SubscribeStrategy.md │ ├── index.md │ └── options.md ├── logical-analysis-rules │ └── ResolveWriteToStream.md ├── logical-operators │ ├── ContinuousExecutionRelation.md │ ├── Deduplicate.md │ ├── EventTimeWatermark.md │ ├── FlatMapGroupsWithState.md │ ├── StreamingDataSourceV2Relation.md │ ├── StreamingExecutionRelation.md │ ├── StreamingRelation.md │ ├── StreamingRelationV2.md │ ├── WriteToContinuousDataSource.md │ ├── WriteToDataSourceV2.md │ ├── WriteToMicroBatchDataSource.md │ ├── WriteToStream.md │ └── WriteToStreamStatement.md ├── micro-batch-execution │ ├── MicroBatchExecution.md │ ├── MicroBatchWrite.md │ ├── MicroBatchWriterFactory.md │ └── index.md ├── monitoring │ ├── ExecutionStats.md │ ├── MetricsReporter.md │ ├── SinkProgress.md │ ├── SourceProgress.md │ ├── StateOperatorProgress.md │ ├── StreamingQueryListener.md │ ├── StreamingQueryProgress.md │ └── StreamingQueryStatus.md ├── offsets-and-metadata-checkpointing.md ├── operators │ ├── crossJoin.md │ ├── dropDuplicates.md │ ├── explain.md │ ├── flatMapGroupsWithState.md │ ├── groupBy.md │ ├── groupByKey.md │ ├── index.md │ ├── join.md │ ├── joinWith.md │ ├── mapGroupsWithState.md │ ├── window.md │ ├── withWatermark.md │ └── writeStream.md ├── overview.md ├── physical-operators │ ├── EventTimeWatermarkExec.md │ ├── FlatMapGroupsWithStateExec.md │ ├── MicroBatchScanExec.md │ ├── SessionWindowStateStoreRestoreExec.md │ ├── SessionWindowStateStoreSaveExec.md │ ├── StateStoreReader.md │ ├── StateStoreRestoreExec.md │ ├── StateStoreSaveExec.md │ ├── StateStoreWriter.md │ ├── StatefulOpClusteredDistribution.md │ ├── StatefulOperator.md │ ├── StreamingDeduplicateExec.md │ ├── StreamingGlobalLimitExec.md │ ├── StreamingRelationExec.md │ ├── StreamingSymmetricHashJoinExec.md │ ├── WatermarkSupport.md │ ├── WriteToContinuousDataSourceExec.md │ └── WriteToDataSourceV2Exec.md ├── rocksdb │ ├── RocksDB.md │ ├── RocksDBConf.md │ ├── RocksDBFileManager.md │ ├── RocksDBMetrics.md │ ├── RocksDBStateStore.md │ ├── RocksDBStateStoreProvider.md │ └── index.md ├── spark-logging.md ├── stateful-stream-processing │ ├── BaseStateStoreRDD.md │ ├── HDFSBackedStateStore.md │ ├── HDFSBackedStateStoreProvider.md │ ├── ReadStateStore.md │ ├── ReadStateStoreRDD.md │ ├── StateSchemaCompatibilityChecker.md │ ├── StateStore.md │ ├── StateStoreConf.md │ ├── StateStoreCoordinator.md │ ├── StateStoreCoordinatorRef.md │ ├── StateStoreCustomMetric.md │ ├── StateStoreId.md │ ├── StateStoreMetrics.md │ ├── StateStoreOps.md │ ├── StateStoreProvider.md │ ├── StateStoreProviderId.md │ ├── StateStoreRDD.md │ ├── StatefulOperatorPartitioning.md │ ├── StatefulOperatorStateInfo.md │ ├── StreamingSessionWindowStateManager.md │ └── index.md ├── streaming-aggregation │ ├── StreamingAggregationStateManager.md │ ├── StreamingAggregationStateManagerBaseImpl.md │ ├── StreamingAggregationStateManagerImplV2.md │ └── index.md ├── streaming-deduplication │ └── index.md ├── streaming-limit │ └── index.md ├── testing.md ├── watermark │ ├── EventTimeStats.md │ ├── EventTimeStatsAccum.md │ ├── WatermarkTracker.md │ └── index.md └── webui │ ├── StreamingQueryPage.md │ ├── StreamingQueryStatisticsPage.md │ ├── StreamingQueryStatusListener.md │ ├── StreamingQueryStatusStore.md │ ├── StreamingQueryTab.md │ └── index.md ├── examples ├── .gitignore ├── README.md ├── build.sbt ├── project │ └── build.properties ├── src │ └── main │ │ ├── resources │ │ ├── KafkaSourceExample.scala │ │ └── log4j.properties │ │ └── scala │ │ └── pl │ │ └── japila │ │ └── spark │ │ ├── Event.scala │ │ ├── FlatMapGroupsWithStateApp.scala │ │ ├── GroupbyAppendWatermarkExample.scala │ │ ├── Session.scala │ │ ├── SparkStreamsApp.scala │ │ ├── StreamBatchJoinApp.scala │ │ ├── StreamStreamJoinApp.scala │ │ └── StreamingAggregationApp.scala └── streaming-source │ ├── README.md │ └── src │ └── main │ ├── resources │ ├── META-INF │ │ └── services │ │ │ └── org.apache.spark.sql.sources.DataSourceRegister │ └── log4j.properties │ └── scala │ ├── demo │ ├── DemoSource.scala │ ├── DemoSourceProvider.scala │ └── Main.scala │ └── org │ └── apache │ └── spark │ └── sql │ └── UsePrivateSqlHack.scala ├── graffles ├── ContinuousExecution-addOffset.graffle ├── DataStreamReader-SparkSession-StreamingRelation.graffle ├── KafkaOffsetReader-fetchSpecificOffsets.graffle ├── KafkaSource-creating-instance.graffle ├── KafkaSource-initialPartitionOffsets.graffle ├── KeyToNumValuesStore-KeyWithIndexToValueStore.graffle ├── MicroBatchExecution.graffle ├── OneSideHashJoiner.graffle ├── ProgressReporter-finishTrigger-timestamps.graffle ├── ProgressReporter-updateProgress.graffle ├── StateStoreRDD-SparkPlans-LogicalPlans-operators.graffle ├── StateStoreRDD-StateStoreCoordinator.graffle ├── StateStoreRestoreExec-IncrementalExecution.graffle ├── StateStoreRestoreExec-StatefulAggregationStrategy.graffle ├── StateStoreSaveExec-IncrementalExecution.graffle ├── StateStoreSaveExec-StateStoreRDD.graffle ├── StateStoreSaveExec-StatefulAggregationStrategy.graffle ├── StreamExecution-creating-instance.graffle ├── StreamExecution-durationMs.graffle ├── StreamExecution-runBatch-addBatch.graffle ├── StreamExecution-runBatch-getBatch.graffle ├── StreamExecution-runBatch-newBatchesPlan.graffle ├── StreamExecution-runBatch-nextBatch.graffle ├── StreamExecution-runBatch-queryPlanning.graffle ├── StreamExecution-runBatches.graffle ├── StreamExecution-start.graffle ├── StreamExecution-uniqueSources.graffle ├── StreamingDeduplicateExec-StreamingDeduplicationStrategy.graffle ├── StreamingExecutionRelation.graffle ├── StreamingQueryListener-onQueryProgress.graffle ├── StreamingQueryListener-onQueryStarted.graffle ├── StreamingQueryListener-onQueryTerminated.graffle ├── StreamingQueryListenerBus.graffle ├── StreamingQueryManager-createQuery.graffle ├── StreamingQueryManager-notifyQueryTermination.graffle ├── StreamingQueryManager-postListenerEvent.graffle ├── StreamingQueryManager.graffle ├── StreamingRelation.graffle ├── StreamingRelationStrategy-apply.graffle └── SymmetricHashJoinStateManager.graffle ├── mkdocs.yml └── requirements.txt /.github/FUNDING.yml: -------------------------------------------------------------------------------- 1 | github: jaceklaskowski 2 | ko_fi: jaceklaskowski 3 | custom: "https://paypal.me/JacekLaskowski" 4 | -------------------------------------------------------------------------------- /.github/workflows/deploying-to-github-pages.yml: -------------------------------------------------------------------------------- 1 | # Based on https://github.com/squidfunk/mkdocs-material/blob/master/.github/workflows/ci.yml 2 | 3 | name: Deploying to GitHub Pages 4 | on: 5 | push: 6 | branches: 7 | - main 8 | 9 | jobs: 10 | deploy: 11 | if: github.event.pull_request.head.repo.fork == false 12 | runs-on: ubuntu-latest 13 | steps: 14 | - uses: actions/checkout@v3 15 | with: 16 | fetch-depth: 0 17 | - uses: actions/setup-python@v3 18 | with: 19 | python-version: 3.x 20 | - name: Install dependencies 21 | env: 22 | GH_TOKEN: ${{ secrets.GH_TOKEN }} 23 | run: | 24 | pip install -r requirements.txt 25 | - name: Build documentation 26 | env: 27 | GOOGLE_ANALYTICS_KEY: ${{ secrets.GOOGLE_ANALYTICS_KEY }} 28 | run: | 29 | mkdocs gh-deploy --force 30 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | site/ 2 | 3 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # The Internals of Spark Structured Streaming Online Book 2 | 3 | [![Deploying to GitHub Pages](https://github.com/japila-books/spark-structured-streaming-internals/actions/workflows/deploying-to-github-pages.yml/badge.svg)](https://github.com/japila-books/spark-structured-streaming-internals/actions/workflows/deploying-to-github-pages.yml) 4 | 5 | The project contains the sources of [The Internals of Spark Structured Streaming](https://books.japila.pl/spark-structured-streaming-internals/) online book. 6 | -------------------------------------------------------------------------------- /docs/AcceptsLatestSeenOffset.md: -------------------------------------------------------------------------------- 1 | # AcceptsLatestSeenOffset 2 | 3 | `AcceptsLatestSeenOffset` is an [extension](#contract) of the [SparkDataStream](SparkDataStream.md) abstraction for [data streams](#implementations) that [setLatestSeenOffset](#setLatestSeenOffset). 4 | 5 | !!! info "Spark 3.3.0" 6 | `AcceptsLatestSeenOffset` is a new feature in [Spark 3.3.0]({{ spark.jira }}/SPARK-37970) with no [implementations](#implementations) yet. 7 | 8 | ## Contract 9 | 10 | ### setLatestSeenOffset 11 | 12 | ```java 13 | void setLatestSeenOffset( 14 | Offset offset) 15 | ``` 16 | 17 | ## Implementations 18 | 19 | !!! note 20 | No built-in implementations available. 21 | -------------------------------------------------------------------------------- /docs/AcceptsLatestSeenOffsetHandler.md: -------------------------------------------------------------------------------- 1 | # AcceptsLatestSeenOffsetHandler 2 | 3 | `AcceptsLatestSeenOffsetHandler` is a utility that can [setLatestSeenOffsetOnSources](#setLatestSeenOffsetOnSources) on [SparkDataStream](SparkDataStream.md)s with support for [AcceptsLatestSeenOffset](AcceptsLatestSeenOffset.md). 4 | 5 | ## setLatestSeenOffsetOnSources 6 | 7 | ```scala 8 | setLatestSeenOffsetOnSources( 9 | offsets: Option[OffsetSeq], 10 | sources: Seq[SparkDataStream]): Unit 11 | ``` 12 | 13 | `setLatestSeenOffsetOnSources`...FIXME 14 | 15 | --- 16 | 17 | `setLatestSeenOffsetOnSources` is used when: 18 | 19 | * `MicroBatchExecution` is requested to [run an activated streaming query](micro-batch-execution/MicroBatchExecution.md#runActivatedStream) (and the [currentBatchId](StreamExecution.md#currentBatchId) has not been initialized yet) 20 | * `ContinuousExecution` is requested to [run a streaming query in continuous mode](continuous-execution/ContinuousExecution.md#runContinuous) (and the [currentBatchId](StreamExecution.md#currentBatchId) has been initialized already) 21 | -------------------------------------------------------------------------------- /docs/AvailableNowDataStreamWrapper.md: -------------------------------------------------------------------------------- 1 | # AvailableNowDataStreamWrapper 2 | 3 | `AvailableNowDataStreamWrapper` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/CommitLog.md: -------------------------------------------------------------------------------- 1 | # CommitLog — HDFSMetadataLog for Offset Commit Log 2 | 3 | `CommitLog` is an [HDFSMetadataLog](HDFSMetadataLog.md) with [CommitMetadata](#CommitMetadata) metadata. 4 | 5 | `CommitLog` is the [offset commit log](StreamExecution.md#commitLog) of [streaming query execution engines](StreamExecution.md). 6 | 7 | [[CommitMetadata]][[nextBatchWatermarkMs]] 8 | `CommitLog` uses `CommitMetadata` for the metadata with *nextBatchWatermarkMs* attribute (of type `Long` and the default `0`). 9 | 10 | `CommitLog` <> commit metadata to files with names that are offsets. 11 | 12 | ```text 13 | $ ls -tr [checkpoint-directory]/commits 14 | 0 1 2 3 4 5 6 7 8 9 15 | 16 | $ cat [checkpoint-directory]/commits/8 17 | v1 18 | {"nextBatchWatermarkMs": 0} 19 | ``` 20 | 21 | [[VERSION]] 22 | `CommitLog` uses *1* for the version. 23 | 24 | [[creating-instance]] 25 | `CommitLog` (like the parent [HDFSMetadataLog](HDFSMetadataLog.md#creating-instance)) takes the following to be created: 26 | 27 | * [[sparkSession]] `SparkSession` 28 | * [[path]] Path of the metadata log directory 29 | 30 | === [[serialize]] Serializing Metadata (Writing Metadata to Persistent Storage) -- `serialize` Method 31 | 32 | [source, scala] 33 | ---- 34 | serialize( 35 | metadata: CommitMetadata, 36 | out: OutputStream): Unit 37 | ---- 38 | 39 | `serialize` writes out the <> prefixed with `v` on a single line (e.g. `v1`) followed by the given `CommitMetadata` in JSON format. 40 | 41 | `serialize` is part of [HDFSMetadataLog](HDFSMetadataLog.md#serialize) abstraction. 42 | 43 | === [[deserialize]] Deserializing Metadata -- `deserialize` Method 44 | 45 | [source, scala] 46 | ---- 47 | deserialize(in: InputStream): CommitMetadata 48 | ---- 49 | 50 | `deserialize` simply reads (_deserializes_) two lines from the given `InputStream` for [version](HDFSMetadataLog.md#parseVersion) and the <> attribute. 51 | 52 | `deserialize` is part of [HDFSMetadataLog](HDFSMetadataLog.md#deserialize) abstraction. 53 | 54 | === [[add-batchId]] `add` Method 55 | 56 | [source, scala] 57 | ---- 58 | add(batchId: Long): Unit 59 | ---- 60 | 61 | `add`...FIXME 62 | 63 | NOTE: `add` is used when...FIXME 64 | 65 | === [[add-batchId-metadata]] `add` Method 66 | 67 | [source, scala] 68 | ---- 69 | add(batchId: Long, metadata: String): Boolean 70 | ---- 71 | 72 | `add`...FIXME 73 | 74 | `add` is part of [MetadataLog](MetadataLog.md#add) abstraction. 75 | -------------------------------------------------------------------------------- /docs/CommitMetadata.md: -------------------------------------------------------------------------------- 1 | == [[CommitMetadata]] CommitMetadata 2 | 3 | `CommitMetadata` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ContinuousDataSourceRDD.md: -------------------------------------------------------------------------------- 1 | # ContinuousDataSourceRDD 2 | 3 | `ContinuousDataSourceRDD` is a specialized `RDD` (`RDD[InternalRow]`) that is used exclusively for the only input RDD (with the input rows) of `DataSourceV2ScanExec` leaf physical operator with a [ContinuousReader](continuous-execution/ContinuousReader.md). 4 | 5 | `ContinuousDataSourceRDD` is <> exclusively when `DataSourceV2ScanExec` leaf physical operator is requested for the input RDDs (which there is only one actually). 6 | 7 | [[spark.sql.streaming.continuous.executorQueueSize]] 8 | `ContinuousDataSourceRDD` uses [spark.sql.streaming.continuous.executorQueueSize](configuration-properties.md#spark.sql.streaming.continuous.executorQueueSize) configuration property for the <>. 9 | 10 | [[spark.sql.streaming.continuous.executorPollIntervalMs]] 11 | `ContinuousDataSourceRDD` uses [spark.sql.streaming.continuous.executorPollIntervalMs](configuration-properties.md#spark.sql.streaming.continuous.executorPollIntervalMs) configuration property for the <>. 12 | 13 | [[creating-instance]] 14 | `ContinuousDataSourceRDD` takes the following to be created: 15 | 16 | * [[sc]] `SparkContext` 17 | * [[dataQueueSize]] Size of the data queue 18 | * [[epochPollIntervalMs]] `epochPollIntervalMs` 19 | * [[readerInputPartitions]] ``InputPartition[InternalRow]``s 20 | 21 | [[getPreferredLocations]] 22 | `ContinuousDataSourceRDD` uses `InputPartition` (of a `ContinuousDataSourceRDDPartition`) for preferred host locations (where the input partition reader can run faster). 23 | 24 | === [[compute]] Computing Partition -- `compute` Method 25 | 26 | [source, scala] 27 | ---- 28 | compute( 29 | split: Partition, 30 | context: TaskContext): Iterator[InternalRow] 31 | ---- 32 | 33 | NOTE: `compute` is part of the RDD Contract to compute a given partition. 34 | 35 | `compute`...FIXME 36 | 37 | === [[getPartitions]] `getPartitions` Method 38 | 39 | [source, scala] 40 | ---- 41 | getPartitions: Array[Partition] 42 | ---- 43 | 44 | NOTE: `getPartitions` is part of the `RDD` Contract to specify the partitions to <>. 45 | 46 | `getPartitions`...FIXME 47 | -------------------------------------------------------------------------------- /docs/ContinuousStream.md: -------------------------------------------------------------------------------- 1 | # ContinuousStream 2 | 3 | `ContinuousStream` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/DataReaderThread.md: -------------------------------------------------------------------------------- 1 | == [[DataReaderThread]] DataReaderThread 2 | 3 | `DataReaderThread` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/EpochCoordinatorRef.md: -------------------------------------------------------------------------------- 1 | == [[EpochCoordinatorRef]] EpochCoordinatorRef 2 | 3 | `EpochCoordinatorRef` is...FIXME 4 | 5 | === [[create]] Creating Remote Reference to EpochCoordinator RPC Endpoint -- `create` Factory Method 6 | 7 | [source, scala] 8 | ---- 9 | create( 10 | writer: StreamWriter, 11 | reader: ContinuousReader, 12 | query: ContinuousExecution, 13 | epochCoordinatorId: String, 14 | startEpoch: Long, 15 | session: SparkSession, 16 | env: SparkEnv): RpcEndpointRef 17 | ---- 18 | 19 | `create`...FIXME 20 | 21 | NOTE: `create` is used exclusively when `ContinuousExecution` is requested to <>. 22 | 23 | === [[get]] Getting Remote Reference to EpochCoordinator RPC Endpoint -- `get` Factory Method 24 | 25 | [source, scala] 26 | ---- 27 | get(id: String, env: SparkEnv): RpcEndpointRef 28 | ---- 29 | 30 | `get`...FIXME 31 | 32 | [NOTE] 33 | ==== 34 | `get` is used when: 35 | 36 | * `DataSourceV2ScanExec` leaf physical operator is requested for the input RDDs (and creates a <> for a [ContinuousReader](continuous-execution/ContinuousReader.md)) 37 | 38 | * `ContinuousQueuedDataReader` is created (and initializes the <>) 39 | 40 | * `EpochMarkerGenerator` is created (and initializes the <>) 41 | 42 | * `ContinuousWriteRDD` is requested to <> 43 | 44 | * `WriteToContinuousDataSourceExec` is requested to <> 45 | ==== 46 | -------------------------------------------------------------------------------- /docs/EpochMarkerGenerator.md: -------------------------------------------------------------------------------- 1 | == [[EpochMarkerGenerator]] EpochMarkerGenerator Thread 2 | 3 | `EpochMarkerGenerator` is...FIXME 4 | 5 | === [[run]] `run` Method 6 | 7 | [source, scala] 8 | ---- 9 | run(): Unit 10 | ---- 11 | 12 | NOTE: `run` is part of the https://docs.oracle.com/javase/8/docs/api/java/lang/Runnable.html[java.lang.Runnable] Contract to be executed upon starting a thread. 13 | 14 | `run`...FIXME 15 | -------------------------------------------------------------------------------- /docs/EpochTracker.md: -------------------------------------------------------------------------------- 1 | == [[EpochTracker]] EpochTracker 2 | 3 | `EpochTracker` is...FIXME 4 | 5 | === [[getCurrentEpoch]] Current Epoch -- `getCurrentEpoch` Method 6 | 7 | [source, scala] 8 | ---- 9 | getCurrentEpoch: Option[Long] 10 | ---- 11 | 12 | `getCurrentEpoch`...FIXME 13 | 14 | NOTE: `getCurrentEpoch` is used when...FIXME 15 | 16 | === [[incrementCurrentEpoch]] Advancing (Incrementing) Epoch -- `incrementCurrentEpoch` Method 17 | 18 | [source, scala] 19 | ---- 20 | incrementCurrentEpoch(): Unit 21 | ---- 22 | 23 | `incrementCurrentEpoch`...FIXME 24 | 25 | NOTE: `incrementCurrentEpoch` is used when...FIXME 26 | -------------------------------------------------------------------------------- /docs/FileContextBasedCheckpointFileManager.md: -------------------------------------------------------------------------------- 1 | # FileContextBasedCheckpointFileManager 2 | 3 | `FileContextBasedCheckpointFileManager` is a [CheckpointFileManager](CheckpointFileManager.md) that uses `FileContext` ([Apache Hadoop]({{ hadoop.api }}/org/apache/hadoop/fs/FileContext.html)). 4 | 5 | `FileContextBasedCheckpointFileManager` is the default [CheckpointFileManager](CheckpointFileManager.md) (unless [spark.sql.streaming.checkpointFileManagerClass](configuration-properties.md#spark.sql.streaming.checkpointFileManagerClass) is defined) as HDFS's [FileContext.rename()]({{ hadoop.api }}/org/apache/hadoop/fs/FileContext.html#rename-org.apache.hadoop.fs.Path-org.apache.hadoop.fs.Path-org.apache.hadoop.fs.Options.Rename...-) gives atomic renames, which is used for [createAtomic](#createAtomic). 6 | 7 | ## Creating Instance 8 | 9 | `FileContextBasedCheckpointFileManager` takes the following to be created: 10 | 11 | * `Path` ([Apache Hadoop]({{ hadoop.api }}/org/apache/hadoop/fs/Path.html)) 12 | * `Configuration` ([Apache Hadoop]({{ hadoop.api }}/org/apache/hadoop/conf/Configuration.html)) 13 | 14 | `FileContextBasedCheckpointFileManager` is created when: 15 | 16 | * `CheckpointFileManager` is requested to [create a CheckpointFileManager](CheckpointFileManager.md#create) (and [spark.sql.streaming.checkpointFileManagerClass](configuration-properties.md#spark.sql.streaming.checkpointFileManagerClass) is not defined) 17 | 18 | ## createAtomic 19 | 20 | ```scala 21 | createAtomic( 22 | path: Path, 23 | overwriteIfPossible: Boolean): CancellableFSDataOutputStream 24 | ``` 25 | 26 | `createAtomic` is part of the [CheckpointFileManager](CheckpointFileManager.md#createAtomic) abstraction. 27 | 28 | --- 29 | 30 | `createAtomic` creates a `RenameBasedFSDataOutputStream`. 31 | -------------------------------------------------------------------------------- /docs/GroupStateTimeout.md: -------------------------------------------------------------------------------- 1 | # GroupStateTimeout 2 | 3 | `GroupStateTimeout` represents an **aggregation state timeout** that defines when a [GroupState](GroupState.md) can be considered **timed-out** (_expired_) in [Arbitrary Stateful Streaming Aggregation](arbitrary-stateful-streaming-aggregation/index.md). 4 | 5 | `GroupStateTimeout` is used with the following `KeyValueGroupedDataset` operations: 6 | 7 | * `mapGroupsWithState` 8 | * `flatMapGroupsWithState` 9 | 10 | [[extensions]] 11 | .GroupStateTimeouts 12 | [cols="30m,70",options="header",width="100%"] 13 | |=== 14 | | GroupStateTimeout 15 | | Description 16 | 17 | | EventTimeTimeout 18 | | [[EventTimeTimeout]] Timeout based on event time 19 | 20 | Used when...FIXME 21 | 22 | | NoTimeout 23 | | [[NoTimeout]] No timeout 24 | 25 | Used when...FIXME 26 | 27 | | ProcessingTimeTimeout 28 | a| [[ProcessingTimeTimeout]] Timeout based on processing time 29 | 30 | [FlatMapGroupsWithStateExec](physical-operators/FlatMapGroupsWithStateExec.md) physical operator requires that `batchTimestampMs` is specified when `ProcessingTimeTimeout` is used. 31 | 32 | `batchTimestampMs` is defined when [IncrementalExecution](IncrementalExecution.md) is created (with the [state](IncrementalExecution.md#state)). `IncrementalExecution` is given `OffsetSeqMetadata` when `StreamExecution` is requested to [run a streaming batch](micro-batch-execution/MicroBatchExecution.md#runBatch). 33 | 34 | |=== 35 | -------------------------------------------------------------------------------- /docs/MetadataLog.md: -------------------------------------------------------------------------------- 1 | # MetadataLog 2 | 3 | `MetadataLog` is an [abstraction](#contract) of [metadata logs](#implementations) that can [add](#add), [get](#get), [getLatest](#getLatest) and [purge](#purge) metadata (of type `T`). 4 | 5 | ??? note "Type Constructor" 6 | `MetadataLog[T]` is a Scala type constructor with the type parameter `T` 7 | 8 | ## Contract 9 | 10 | ###  Storing Metadata of Streaming Batch 11 | 12 | ```scala 13 | add( 14 | batchId: Long, 15 | metadata: T): Boolean 16 | ``` 17 | 18 | Stores (_adds_) metadata of a streaming batch 19 | 20 | Used when: 21 | 22 | * `KafkaSource` is requested for the [initialPartitionOffsets](kafka/KafkaSource.md#initialPartitionOffsets) 23 | 24 | * `CompactibleFileStreamLog` is requested for the [store metadata of a streaming batch](datasources/file/CompactibleFileStreamLog.md#add) and to [compact](datasources/file/CompactibleFileStreamLog.md#compact) 25 | 26 | * `FileStreamSource` is requested to [fetchMaxOffset](datasources/file/FileStreamSource.md#fetchMaxOffset) 27 | 28 | * `FileStreamSourceLog` is requested to [store (add) metadata of a streaming batch](datasources/file/FileStreamSourceLog.md#add) 29 | 30 | * `ManifestFileCommitProtocol` is requested to [commitJob](datasources/file/ManifestFileCommitProtocol.md#commitJob) 31 | 32 | * `MicroBatchExecution` stream execution engine is requested to <> and <> 33 | 34 | * `ContinuousExecution` stream execution engine is requested to <> and <> 35 | 36 | ###  get 37 | 38 | ```scala 39 | get( 40 | batchId: Long): Option[T] 41 | get( 42 | startId: Option[Long], 43 | endId: Option[Long]): Array[(Long, T)] 44 | ``` 45 | 46 | Looks up (_gets_) metadata of one or more streaming batches 47 | 48 | Used when...FIXME 49 | 50 | ###  getLatest 51 | 52 | ```scala 53 | getLatest(): Option[(Long, T)] 54 | ``` 55 | 56 | Looks up the latest-committed metadata (if available) 57 | 58 | Used when...FIXME 59 | 60 | ###  purge 61 | 62 | ```scala 63 | purge( 64 | thresholdBatchId: Long): Unit 65 | ``` 66 | 67 | Purging (_removing_) metadata older than the given threshold 68 | 69 | Used when...FIXME 70 | 71 | ## Implementations 72 | 73 | * [HDFSMetadataLog](HDFSMetadataLog.md) 74 | -------------------------------------------------------------------------------- /docs/MicroBatchStream.md: -------------------------------------------------------------------------------- 1 | # MicroBatchStream 2 | 3 | `MicroBatchStream` is an [extension](#contract) of the [SparkDataStream](SparkDataStream.md) abstraction for [data streams](#implementations) for [Micro-Batch Stream Processing](micro-batch-execution/index.md). 4 | 5 | ## Contract 6 | 7 | ###  Creating PartitionReaderFactory 8 | 9 | ```java 10 | PartitionReaderFactory createReaderFactory() 11 | ``` 12 | 13 | `PartitionReaderFactory` ([Spark SQL]({{ book.spark_sql }}/connector/PartitionReaderFactory)) 14 | 15 | Used when: 16 | 17 | * `MicroBatchScanExec` physical operator is requested for a [PartitionReaderFactory](physical-operators/MicroBatchScanExec.md#readerFactory) 18 | 19 | ###  Latest Offset 20 | 21 | ```java 22 | Offset latestOffset() 23 | ``` 24 | 25 | Latest [Offset](Offset.md) 26 | 27 | Used when: 28 | 29 | * `MicroBatchExecution` is requested to [constructing or skipping next streaming micro-batch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) 30 | 31 | ###  Input Partitions 32 | 33 | ```java 34 | InputPartition[] planInputPartitions( 35 | Offset start, 36 | Offset end) 37 | ``` 38 | 39 | `InputPartition`s ([Spark SQL]({{ book.spark_sql }}/connector/InputPartition)) 40 | 41 | Used when: 42 | 43 | * `MicroBatchScanExec` physical operator is requested for [input partitions](physical-operators/MicroBatchScanExec.md#partitions) 44 | 45 | ## Implementations 46 | 47 | * `AvailableNowMicroBatchStreamWrapper` 48 | * [KafkaMicroBatchStream](kafka/KafkaMicroBatchStream.md) 49 | * [MemoryStream](datasources/memory/MemoryStream.md) 50 | * `RatePerMicroBatchStream` 51 | * `RateStreamMicroBatchStream` 52 | * `TextSocketMicroBatchStream` 53 | -------------------------------------------------------------------------------- /docs/Offset.md: -------------------------------------------------------------------------------- 1 | # Offset — Read Position of Streaming Query 2 | 3 | `Offset` is an [abstraction](#contract) of [stream positions](#implementations). 4 | 5 | !!! note 6 | There are two `Offset` abstractions and new streaming data sources should use Data Source v2 API. 7 | 8 | ## Contract 9 | 10 | ###  JSON Representation 11 | 12 | ```java 13 | String json() 14 | ``` 15 | 16 | JSON-encoded representation of the offset 17 | 18 | Used when: 19 | 20 | * `MicroBatchExecution` stream execution engine is requested to [construct the next streaming micro-batch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) and [run a streaming micro-batch](micro-batch-execution/MicroBatchExecution.md#runBatch) 21 | * `OffsetSeq` is requested for the [textual representation](OffsetSeq.md#toString) 22 | * `OffsetSeqLog` is requested to [serialize metadata (write metadata in serialized format)](OffsetSeqLog.md#serialize) 23 | * `ProgressReporter` is requested to [record trigger offsets](ProgressReporter.md#recordTriggerOffsets) 24 | * `ContinuousExecution` stream execution engine is requested to [run a streaming query in continuous mode](continuous-execution/ContinuousExecution.md#runContinuous) and [commit an epoch](continuous-execution/ContinuousExecution.md#commit) 25 | 26 | ## Implementations 27 | 28 | * ContinuousMemoryStreamOffset 29 | * FileStreamSourceOffset 30 | * [KafkaSourceOffset](kafka/KafkaSourceOffset.md) 31 | * LongOffset 32 | * SerializedOffset 33 | * TextSocketOffset 34 | -------------------------------------------------------------------------------- /docs/OutputMode.md: -------------------------------------------------------------------------------- 1 | # OutputMode 2 | 3 | `OutputMode` of a streaming query described what data is written to a [streaming sink](Sink.md). 4 | 5 | `OutputMode` is specified using [DataStreamWriter.outputMode](DataStreamWriter.md#outputMode). 6 | 7 | ## Output Modes 8 | 9 | ### Append 10 | 11 | **Append** (alias: `append`) is the [default output mode](DataStreamWriter.md#outputMode) that writes "new" rows only. 12 | 13 | In [streaming aggregations](streaming-aggregation/index.md), a "new" row is when the intermediate state becomes final, i.e. when new events for the grouping key can only be considered late which is when watermark moves past the event time of the key. 14 | 15 | `Append` output mode requires that a streaming query defines event-time watermark (using [withWatermark](operators/withWatermark.md) operator) on the event time column that is used in aggregation (directly or using [window](operators/window.md) standard function). 16 | 17 | Required for datasets with `FileFormat` format (to create [FileStreamSink](datasources/file/FileStreamSink.md)) 18 | 19 | `Append` is [mandatory](UnsupportedOperationChecker.md#multiple-flatMapGroupsWithState) when multiple `flatMapGroupsWithState` operators are used in a structured query. 20 | 21 | ### Complete 22 | 23 | **Complete** (alias: `complete`) writes all the rows of a Result Table (and corresponds to a traditional batch structured query). 24 | 25 | Complete mode does not drop old aggregation state and preserves all data in the Result Table. 26 | 27 | Supported only for [streaming aggregations](streaming-aggregation/index.md) (as asserted by [UnsupportedOperationChecker](UnsupportedOperationChecker.md#checkForStreaming)). 28 | 29 | ### Update 30 | 31 | **Update** (alias: `update`) writes only the rows that were updated (every time there are updates). 32 | 33 | For queries that are not [streaming aggregations](streaming-aggregation/index.md), `Update` is equivalent to the [Append](#Append) output mode. 34 | -------------------------------------------------------------------------------- /docs/PartitionOffset.md: -------------------------------------------------------------------------------- 1 | == [[PartitionOffset]] PartitionOffset 2 | 3 | `PartitionOffset` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ProcessingTimeExecutor.md: -------------------------------------------------------------------------------- 1 | # ProcessingTimeExecutor 2 | 3 | `ProcessingTimeExecutor` is a [TriggerExecutor](TriggerExecutor.md) that [executes a trigger handler](#execute) every [ProcessingTimeTrigger](#processingTimeTrigger) (until the handler returns `false`). 4 | 5 | `ProcessingTimeExecutor` is created for the following [Trigger](Trigger.md)s: 6 | 7 | * [Trigger.ProcessingTime](Trigger.md#ProcessingTimeTrigger) in [Micro-Batch Stream Processing](micro-batch-execution/index.md) 8 | * [Trigger.Continuous](Trigger.md#ContinuousTrigger) in [Continuous Stream Processing](continuous-execution/index.md) 9 | 10 | !!! note "Continuous Stream Processing" 11 | `ProcessingTimeExecutor` is the only [TriggerExecutor](TriggerExecutor.md) supported in [Continuous Stream Processing](continuous-execution/index.md). 12 | 13 | ## Creating Instance 14 | 15 | `ProcessingTimeExecutor` takes the following to be created: 16 | 17 | * [ProcessingTimeTrigger](Trigger.md#ProcessingTimeTrigger) 18 | * `Clock` 19 | 20 | `ProcessingTimeExecutor` is created when: 21 | 22 | * `MicroBatchExecution` is [created](micro-batch-execution/MicroBatchExecution.md#triggerExecutor) (with a [ProcessingTimeTrigger](Trigger.md#ProcessingTimeTrigger)) 23 | * `ContinuousExecution` is [created](continuous-execution/ContinuousExecution.md#triggerExecutor) (with a [ContinuousTrigger](Trigger.md#ContinuousTrigger)) 24 | 25 | ## Executing Trigger 26 | 27 | ```scala 28 | execute( 29 | triggerHandler: () => Boolean): Unit 30 | ``` 31 | 32 | `execute` is part of the [TriggerExecutor](TriggerExecutor.md#execute) abstraction. 33 | 34 | --- 35 | 36 | `execute` [calculates the next batch's start time](#nextBatchTime). 37 | 38 | `execute` executes the given `triggerHandler` every [processingTimeTrigger](#processingTimeTrigger) until the given `triggerHandler` returns `false` (to signal execution termination). 39 | 40 | ### Next Batch's Start Time 41 | 42 | ```scala 43 | nextBatchTime( 44 | now: Long): Long 45 | ``` 46 | 47 | `nextBatchTime` returns the start time (in millis) of the next batch interval given the current `now` time. 48 | -------------------------------------------------------------------------------- /docs/ReportsSourceMetrics.md: -------------------------------------------------------------------------------- 1 | # ReportsSourceMetrics 2 | 3 | `ReportsSourceMetrics` is an [extension](#contract) of the [SparkDataStream](SparkDataStream.md) abstraction for [data streams](#implementations) with [metrics](#metrics). 4 | 5 | ## Contract 6 | 7 | ### Performance Metrics 8 | 9 | ```java 10 | Map metrics( 11 | Optional latestConsumedOffset) 12 | ``` 13 | 14 | Metrics of this [SparkDataStream](SparkDataStream.md) for the latest consumed offset (to create a [SourceProgress](monitoring/SourceProgress.md#metrics) for a [StreamingQueryProgress](monitoring/StreamingQueryProgress.md#sources)) 15 | 16 | See [KafkaMicroBatchStream](kafka/KafkaMicroBatchStream.md#metrics) 17 | 18 | Used when: 19 | 20 | * `ProgressReporter` is requested to [finish up a micro-batch](ProgressReporter.md#finishTrigger) 21 | 22 | ## Implementations 23 | 24 | * [KafkaMicroBatchStream](kafka/KafkaMicroBatchStream.md) 25 | -------------------------------------------------------------------------------- /docs/Sink.md: -------------------------------------------------------------------------------- 1 | # Sink 2 | 3 | `Sink` is an [extension](#contract) of the `Table` abstraction for [streaming sinks](#implementations) that [add the batch results of a streaming query](#addBatch) in [Micro-Batch Stream Processing](micro-batch-execution/index.md). 4 | 5 | !!! note 6 | `Sink` extends `Table` interface for the only purpose of making it compatible with Data Source V2. All `Table` methods simply throw an `IllegalStateException`. 7 | 8 | ## Contract 9 | 10 | ###  Adding Batch 11 | 12 | ```scala 13 | addBatch( 14 | batchId: Long, 15 | data: DataFrame): Unit 16 | ``` 17 | 18 | Adds a batch of data to the sink 19 | 20 | Used when `MicroBatchExecution` stream execution engine is requested to [add a batch to a sink (addBatch phase)](micro-batch-execution/MicroBatchExecution.md#runBatch-addBatch) (while [running micro-batches of a streaming query](micro-batch-execution/MicroBatchExecution.md#runBatch)) 21 | 22 | ## Implementations 23 | 24 | * [FileStreamSink](datasources/file/FileStreamSink.md) 25 | * [ForeachBatchSink](datasources/ForeachBatchSink.md) 26 | * [KafkaSink](kafka/KafkaSink.md) 27 | -------------------------------------------------------------------------------- /docs/SparkDataStream.md: -------------------------------------------------------------------------------- 1 | # SparkDataStream 2 | 3 | `SparkDataStream` is an [abstraction](#contract) of [readable data streams](#implementations). 4 | 5 | ## Contract 6 | 7 | ###  commit 8 | 9 | ```java 10 | void commit( 11 | Offset end) 12 | ``` 13 | 14 | Used when: 15 | 16 | * `ContinuousExecution` stream execution engine is requested to [commit](continuous-execution/ContinuousExecution.md#commit) 17 | * `MicroBatchExecution` stream execution engine is requested to [constructNextBatch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) 18 | 19 | ###  deserializeOffset 20 | 21 | ```java 22 | Offset deserializeOffset( 23 | String json) 24 | ``` 25 | 26 | Used when: 27 | 28 | * `ContinuousExecution` stream execution engine is requested to [runContinuous](continuous-execution/ContinuousExecution.md#runContinuous) and [commit](continuous-execution/ContinuousExecution.md#commit) 29 | * `MicroBatchExecution` stream execution engine is requested to [constructNextBatch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) and [runBatch](micro-batch-execution/MicroBatchExecution.md#runBatch) 30 | 31 | ###  initialOffset 32 | 33 | ```java 34 | Offset initialOffset() 35 | ``` 36 | 37 | Used when: 38 | 39 | * `ContinuousExecution` stream execution engine is requested to [runContinuous](continuous-execution/ContinuousExecution.md#runContinuous) 40 | * `MicroBatchExecution` stream execution engine is requested to [constructNextBatch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) and [runBatch](micro-batch-execution/MicroBatchExecution.md#runBatch) 41 | 42 | ###  stop 43 | 44 | ```java 45 | void stop() 46 | ``` 47 | 48 | Used when: 49 | 50 | * `StreamExecution` is requested to [stop sources](StreamExecution.md#stopSources) 51 | 52 | ## Implementations 53 | 54 | * [ContinuousStream](ContinuousStream.md) 55 | * [MemoryStreamBase](datasources/memory/MemoryStreamBase.md) 56 | * [MicroBatchStream](MicroBatchStream.md) 57 | * [Source](Source.md) 58 | * [SupportsAdmissionControl](SupportsAdmissionControl.md) 59 | -------------------------------------------------------------------------------- /docs/StreamProgress.md: -------------------------------------------------------------------------------- 1 | # StreamProgress 2 | 3 | `StreamProgress` is an immutable `Map` ([Scala]({{ scala.api }}/scala/collection/immutable/Map.html)) of [Offset](Offset.md)s by [SparkDataStream](SparkDataStream.md) (`Map[SparkDataStream, OffsetV2]`) 4 | 5 | ```scala 6 | class StreamProgress(...) 7 | extends Map[SparkDataStream, OffsetV2] 8 | ``` 9 | 10 | In other words, `StreamProgress` represents [source data stream](SparkDataStream.md)s of a streaming query with their [position](Offset.md). 11 | 12 | ## Creating Instance 13 | 14 | `StreamProgress` takes the following to be created: 15 | 16 | * Base Map (default: empty) 17 | 18 | `StreamProgress` is created when: 19 | 20 | * `OffsetSeq` is requested to [toStreamProgress](OffsetSeq.md#toStreamProgress) 21 | * [StreamExecution](StreamExecution.md) is created (and creates [committed](StreamExecution.md#committedOffsets), [available](StreamExecution.md#availableOffsets), [latest](StreamExecution.md#latestOffsets) offset trackers) 22 | * `StreamProgress` is requested to [++](#concat) 23 | 24 | ## toOffsetSeq 25 | 26 | ```scala 27 | toOffsetSeq( 28 | source: Seq[SparkDataStream], 29 | metadata: OffsetSeqMetadata): OffsetSeq 30 | ``` 31 | 32 | `toOffsetSeq` creates an [OffsetSeq](OffsetSeq.md). 33 | 34 | --- 35 | 36 | `toOffsetSeq` is used when: 37 | 38 | * `StreamExecution` is requested to [runStream](StreamExecution.md#runStream) (to create a `StreamingQueryException` when a streaming query fails) 39 | -------------------------------------------------------------------------------- /docs/StreamSinkProvider.md: -------------------------------------------------------------------------------- 1 | # StreamSinkProvider 2 | 3 | `StreamSinkProvider` is the <> of <> that can <> for a file format (e.g. `parquet`) or system (e.g. `kafka`). 4 | 5 | [[contract]] 6 | .StreamSinkProvider Contract 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | createSink 13 | a| [[createSink]] 14 | 15 | [source, scala] 16 | ---- 17 | createSink( 18 | sqlContext: SQLContext, 19 | parameters: Map[String, String], 20 | partitionColumns: Seq[String], 21 | outputMode: OutputMode): Sink 22 | ---- 23 | 24 | Creates a [streaming sink](Sink.md) 25 | 26 | Used when `DataSource` is requested for a [streaming sink](DataSource.md#createSink) (when `DataStreamWriter` is requested to [start a streaming query](DataStreamWriter.md#start)) 27 | 28 | |=== 29 | 30 | [[implementations]] 31 | NOTE: [KafkaSourceProvider](kafka/KafkaSourceProvider.md) is the only known `StreamSinkProvider` in Spark Structured Streaming. 32 | -------------------------------------------------------------------------------- /docs/StreamSourceProvider.md: -------------------------------------------------------------------------------- 1 | # StreamSourceProvider 2 | 3 | `StreamSourceProvider` is an [abstraction](#contract) of [data source providers](#implementations) that can [create a streaming source](#createSource) for a data format or system. 4 | 5 | `StreamSourceProvider` is part of Data Source API V1 for [Micro-Batch Stream Processing](micro-batch-execution/index.md). 6 | 7 | ## Contract 8 | 9 | ###  Creating Streaming Source 10 | 11 | ```scala 12 | createSource( 13 | sqlContext: SQLContext, 14 | metadataPath: String, 15 | schema: Option[StructType], 16 | providerName: String, 17 | parameters: Map[String, String]): Source 18 | ``` 19 | 20 | Creates a [streaming Source](Source.md) 21 | 22 | `metadataPath` is the value of the optional user-specified `checkpointLocation` option or resolved by [StreamingQueryManager](StreamingQueryManager.md#createQuery). 23 | 24 | Used when: 25 | 26 | * `DataSource` is requested to [create a streaming source](DataSource.md#createSource) (for a [StreamSourceProvider](StreamSourceProvider.md)) 27 | 28 | ###  Source Schema 29 | 30 | ```scala 31 | sourceSchema( 32 | sqlContext: SQLContext, 33 | schema: Option[StructType], 34 | providerName: String, 35 | parameters: Map[String, String]): (String, StructType) 36 | ``` 37 | 38 | Name and schema of the [Streaming Source](Source.md) 39 | 40 | Used when: 41 | 42 | * `DataSource` is requested for [metadata of a streaming source](DataSource.md#sourceSchema) (when `MicroBatchExecution` is requested to [initialize the analyzed logical plan](micro-batch-execution/MicroBatchExecution.md#logicalPlan)) 43 | 44 | ## Implementations 45 | 46 | * [KafkaSourceProvider](kafka/KafkaSourceProvider.md) 47 | -------------------------------------------------------------------------------- /docs/StreamingDataWriterFactory.md: -------------------------------------------------------------------------------- 1 | # StreamingDataWriterFactory 2 | 3 | `StreamingDataWriterFactory` is an [abstraction](#contract) of [factories](#implementations) to [create a DataWriter](#createWriter). 4 | 5 | ## Contract 6 | 7 | ### Creating DataWriter 8 | 9 | ```java 10 | DataWriter createWriter( 11 | int partitionId, 12 | long taskId, 13 | long epochId) 14 | ``` 15 | 16 | Creates a `DataWriter` ([Spark SQL]({{ book.spark_sql }}/connector/DataWriter)) 17 | 18 | Used when: 19 | 20 | * `ContinuousWriteRDD` is requested to [compute a partition](ContinuousWriteRDD.md#compute) 21 | * `MicroBatchWriterFactory` is requested to [create a DataWriter](micro-batch-execution/MicroBatchWriterFactory.md#createWriter) 22 | 23 | ## Implementations 24 | 25 | * `ForeachWriterFactory` 26 | * [KafkaStreamWriterFactory](kafka/KafkaStreamWriterFactory.md) 27 | * `MemoryWriterFactory` 28 | * `NoopStreamingDataWriterFactory` 29 | * `PackedRowWriterFactory` 30 | -------------------------------------------------------------------------------- /docs/StreamingQueryWrapper.md: -------------------------------------------------------------------------------- 1 | # StreamingQueryWrapper — Serializable StreamExecution 2 | 3 | 4 | `StreamingQueryWrapper` is a serializable interface of a [StreamExecution](StreamExecution.md). 5 | 6 | `StreamingQueryWrapper` has the same [StreamExecution](StreamExecution.md) API and simply passes all the method calls along to the underlying [StreamExecution](#_streamingQuery). 7 | 8 | `StreamingQueryWrapper` is created when `StreamingQueryManager` is requested to [create a streaming query](StreamingQueryManager.md#createQuery) (when `DataStreamWriter` is requested to [start an execution of the streaming query](DataStreamWriter.md#start)). 9 | 10 | ## Demo: Any Streaming Query is StreamingQueryWrapper 11 | 12 | ```scala 13 | import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper 14 | val query = spark 15 | .readStream 16 | .format("rate") 17 | .load 18 | .writeStream 19 | .format("memory") 20 | .queryName("rate2memory") 21 | .start 22 | assert(query.isInstanceOf[StreamingQueryWrapper]) 23 | ``` 24 | -------------------------------------------------------------------------------- /docs/StreamingWrite.md: -------------------------------------------------------------------------------- 1 | # StreamingWrite 2 | 3 | `StreamingWrite` is an [abstraction](#contract) of [streaming writers](#implementations). 4 | 5 | ## Contract 6 | 7 | ### Aborting Writing Job 8 | 9 | ```java 10 | void abort( 11 | long epochId, 12 | WriterCommitMessage[] messages) 13 | ``` 14 | 15 | Used when: 16 | 17 | * `MicroBatchWrite` is requested to [abort](micro-batch-execution/MicroBatchWrite.md#abort) 18 | 19 | ### Committing Writing Job 20 | 21 | ```java 22 | void commit( 23 | long epochId, 24 | WriterCommitMessage[] messages) 25 | ``` 26 | 27 | Used when: 28 | 29 | * `EpochCoordinator` is requested to [commitEpoch](EpochCoordinator.md#commitEpoch) 30 | * `MicroBatchWrite` is requested to [commit](micro-batch-execution/MicroBatchWrite.md#commit) 31 | 32 | ### Creating StreamingDataWriterFactory 33 | 34 | ```java 35 | StreamingDataWriterFactory createStreamingWriterFactory( 36 | PhysicalWriteInfo info) 37 | ``` 38 | 39 | Used when: 40 | 41 | * `MicroBatchWrite` is requested to [createBatchWriterFactory](micro-batch-execution/MicroBatchWrite.md#createBatchWriterFactory) 42 | * `WriteToContinuousDataSourceExec` physical operator is requested to [execute](physical-operators/WriteToContinuousDataSourceExec.md#doExecute) 43 | 44 | ## Implementations 45 | 46 | * `ForeachWrite` 47 | * `ConsoleWrite` 48 | * [KafkaStreamingWrite](kafka/KafkaStreamingWrite.md) 49 | * `MemoryStreamingWrite` 50 | * `NoopStreamingWrite` 51 | -------------------------------------------------------------------------------- /docs/SupportsTriggerAvailableNow.md: -------------------------------------------------------------------------------- 1 | # SupportsTriggerAvailableNow 2 | 3 | `SupportsTriggerAvailableNow` is an [extension](#contract) of the [SupportsAdmissionControl](SupportsAdmissionControl.md) abstraction for [streaming sources](#implementations) to support [Trigger.AvailableNow](Trigger.md#AvailableNow) mode. 4 | 5 | ## Contract 6 | 7 | ### prepareForTriggerAvailableNow 8 | 9 | ```java 10 | void prepareForTriggerAvailableNow() 11 | ``` 12 | 13 | Lets a streaming source to prepare for the [default ReadLimit](SupportsAdmissionControl.md#getDefaultReadLimit) (in [Trigger.AvailableNow](Trigger.md#AvailableNow) mode) 14 | 15 | Used when: 16 | 17 | * `MicroBatchExecution` is requested for the [logicalPlan](micro-batch-execution/MicroBatchExecution.md#logicalPlan) (and the [uniqueSources](micro-batch-execution/MicroBatchExecution.md#uniqueSources) for [MultiBatchExecutor](TriggerExecutor.md#MultiBatchExecutor)) 18 | 19 | ## Implementations 20 | 21 | * [AvailableNowDataStreamWrapper](AvailableNowDataStreamWrapper.md) 22 | * [FileStreamSource](datasources/file/FileStreamSource.md) 23 | * [KafkaMicroBatchStream](kafka/KafkaMicroBatchStream.md) 24 | * [KafkaSource](kafka/KafkaSource.md) 25 | * [RatePerMicroBatchStream](datasources/rate-micro-batch/RatePerMicroBatchStream.md) 26 | -------------------------------------------------------------------------------- /docs/TriggerExecutor.md: -------------------------------------------------------------------------------- 1 | # TriggerExecutor 2 | 3 | `TriggerExecutor` is an [abstraction](#contract) of [trigger executors](#implementations). 4 | 5 | ## Contract 6 | 7 | ### Executing Batches 8 | 9 | ```scala 10 | execute( 11 | batchRunner: () => Boolean): Unit 12 | ``` 13 | 14 | Executes batches (_triggers_) using a batch runner (_trigger handler_). 15 | 16 | `batchRunner` is assumed to return `false` to indicate execution termination 17 | 18 | Used when: 19 | 20 | * `MicroBatchExecution` is requested to [run an activated streaming query](micro-batch-execution/MicroBatchExecution.md#runActivatedStream) 21 | 22 | ## Implementations 23 | 24 | * [MultiBatchExecutor](#MultiBatchExecutor) 25 | * [ProcessingTimeExecutor](ProcessingTimeExecutor.md) 26 | * [SingleBatchExecutor](#SingleBatchExecutor) 27 | 28 | ### MultiBatchExecutor 29 | 30 | Executes the [batch runner](#batchRunner) until it returns `false` 31 | 32 | Handles [AvailableNowTrigger](Trigger.md#AvailableNowTrigger) in [MicroBatchExecution](micro-batch-execution/MicroBatchExecution.md) 33 | 34 | Used when: 35 | 36 | * `MicroBatchExecution` is requested for the [analyzed logical plan](micro-batch-execution/MicroBatchExecution.md#logicalPlan) (and [extracting unique streaming sources](micro-batch-execution/MicroBatchExecution.md#uniqueSources)) 37 | 38 | ### SingleBatchExecutor 39 | 40 | Executes the [batch runner](#batchRunner) exactly once 41 | -------------------------------------------------------------------------------- /docs/arbitrary-stateful-streaming-aggregation/FlatMapGroupsWithStateExecHelper.md: -------------------------------------------------------------------------------- 1 | # FlatMapGroupsWithStateExecHelper 2 | 3 | `FlatMapGroupsWithStateExecHelper` utility is mainly used to [creating a StateManager](#createStateManager) for [FlatMapGroupsWithStateExec](../physical-operators/FlatMapGroupsWithStateExec.md) physical operator. 4 | 5 | === [[createStateManager]] Creating StateManager 6 | 7 | [source, scala] 8 | ---- 9 | createStateManager( 10 | stateEncoder: ExpressionEncoder[Any], 11 | shouldStoreTimestamp: Boolean, 12 | stateFormatVersion: Int): StateManager 13 | ---- 14 | 15 | `createStateManager` simply creates a <> (with the `stateEncoder` and `shouldStoreTimestamp` flag) based on `stateFormatVersion`: 16 | 17 | * <> for `1` 18 | 19 | * <> for `2` 20 | 21 | `createStateManager` throws an `IllegalArgumentException` for `stateFormatVersion` not `1` or `2`: 22 | 23 | ```text 24 | Version [stateFormatVersion] is invalid 25 | ``` 26 | 27 | `createStateManager` is used for the [StateManager](../physical-operators/FlatMapGroupsWithStateExec.md#stateManager) for [FlatMapGroupsWithStateExec](../physical-operators/FlatMapGroupsWithStateExec.md) physical operator. 28 | -------------------------------------------------------------------------------- /docs/arbitrary-stateful-streaming-aggregation/StateManagerImplV1.md: -------------------------------------------------------------------------------- 1 | # StateManagerImplV1 2 | 3 | `StateManagerImplV1` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/arbitrary-stateful-streaming-aggregation/StateManagerImplV2.md: -------------------------------------------------------------------------------- 1 | # StateManagerImplV2 2 | 3 | `StateManagerImplV2` is a concrete [StateManager](StateManager.md) (as a [StateManagerImplBase](StateManagerImplBase.md)) that is used by default in [FlatMapGroupsWithStateExec](../physical-operators/FlatMapGroupsWithStateExec.md) physical operator (per [spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion](../configuration-properties.md#spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion) internal configuration property). 4 | 5 | ## Creating Instance 6 | 7 | `StateManagerImplV2` takes the following to be created: 8 | 9 | * [[stateEncoder]] State encoder (`ExpressionEncoder[Any]`) 10 | * [[shouldStoreTimestamp]] `shouldStoreTimestamp` flag 11 | 12 | `StateManagerImplV2` is created when: 13 | 14 | * `FlatMapGroupsWithStateExecHelper` utility is requested for a [StateManager](FlatMapGroupsWithStateExecHelper.md#createStateManager) (when the `stateFormatVersion` is `2`) 15 | -------------------------------------------------------------------------------- /docs/batch-processing-time.md: -------------------------------------------------------------------------------- 1 | # Batch Processing Time 2 | 3 | **Batch Processing Time** (aka _Batch Timeout Threshold_) is the processing time (_processing timestamp_) of the current streaming batch. 4 | 5 | The following standard functions (and their Catalyst expressions) allow accessing the batch processing time in [Micro-Batch Stream Processing](micro-batch-execution/index.md): 6 | 7 | * `now`, `current_timestamp`, and `unix_timestamp` functions (`CurrentTimestamp`) 8 | 9 | * `current_date` function (`CurrentDate`) 10 | 11 | !!! note 12 | `CurrentTimestamp` or `CurrentDate` expressions are not supported in [Continuous Stream Processing](continuous-execution/index.md). 13 | 14 | ## Internals 15 | 16 | [GroupStateImpl](GroupStateImpl.md) is given the batch processing time when created for a [streaming query](GroupStateImpl.md#createForStreaming) (that is actually the [batch processing time](physical-operators/FlatMapGroupsWithStateExec.md#batchTimestampMs) of the [FlatMapGroupsWithStateExec](physical-operators/FlatMapGroupsWithStateExec.md) physical operator). 17 | 18 | When created, `FlatMapGroupsWithStateExec` physical operator has the processing time undefined and set to the current timestamp in the [state preparation rule](IncrementalExecution.md#state) every streaming batch. 19 | 20 | The current timestamp (and other batch-specific configurations) is given as the [OffsetSeqMetadata](IncrementalExecution.md#offsetSeqMetadata) (as part of the query planning phase) when a [stream execution engine](StreamExecution.md) does the following: 21 | 22 | * `MicroBatchExecution` is requested to [construct a next streaming micro-batch](micro-batch-execution/MicroBatchExecution.md#constructNextBatch) in [Micro-Batch Stream Processing](micro-batch-execution/index.md) 23 | 24 | * In [Continuous Stream Processing](continuous-execution/index.md) the base `StreamExecution` is requested to [run stream processing](StreamExecution.md#runStream) and initializes `OffsetSeqMetadata` to ``0``s. 25 | -------------------------------------------------------------------------------- /docs/continuous-execution/ContinuousReadSupport.md: -------------------------------------------------------------------------------- 1 | # ContinuousReadSupport 2 | 3 | `ContinuousReadSupport` is the <> of the `DataSourceV2` for <> with a <> for [Continuous Stream Processing](index.md). 4 | 5 | [[contract]][[createContinuousReader]] 6 | `ContinuousReadSupport` defines a single `createContinuousReader` method to create a [ContinuousReader](ContinuousReader.md). 7 | 8 | ```scala 9 | ContinuousReader createContinuousReader( 10 | Optional schema, 11 | String checkpointLocation, 12 | DataSourceOptions options) 13 | ``` 14 | 15 | `createContinuousReader` is used when: 16 | 17 | * `ContinuousExecution` is requested to [run a streaming query](ContinuousExecution.md#runContinuous) (and finds [ContinuousExecutionRelations](../logical-operators/ContinuousExecutionRelation.md) in the [analyzed logical plan](ContinuousExecution.md#logicalPlan)) 18 | 19 | * `DataStreamReader` is requested to [create a streaming query for a ContinuousReadSupport data source](../DataStreamReader.md#load) 20 | 21 | [[implementations]] 22 | .ContinuousReadSupports 23 | [cols="30,70",options="header",width="100%"] 24 | |=== 25 | | ContinuousReadSupport 26 | | Description 27 | 28 | | [ContinuousMemoryStream](../datasources/memory/ContinuousMemoryStream.md) 29 | | [[ContinuousMemoryStream]] Data source provider for `memory` format 30 | 31 | | [KafkaSourceProvider](../kafka/KafkaSourceProvider.md) 32 | | [[KafkaSourceProvider]] Data source provider for `kafka` format 33 | 34 | | [RateStreamProvider](../datasources/rate/RateStreamProvider.md) 35 | | [[RateStreamProvider]] Data source provider for `rate` format 36 | 37 | | [TextSocketSourceProvider](../datasources/socket/TextSocketSourceProvider.md) 38 | | [[TextSocketSourceProvider]] Data source provider for `socket` format 39 | 40 | |=== 41 | -------------------------------------------------------------------------------- /docs/datasources/console/ConsoleSinkProvider.md: -------------------------------------------------------------------------------- 1 | # ConsoleSinkProvider 2 | 3 | `ConsoleSinkProvider` is a `SimpleTableProvider` ([Spark SQL]({{ book.spark_sql }}/connector/SimpleTableProvider)) for `console` data source. 4 | 5 | `ConsoleSinkProvider` is a `DataSourceRegister` ([Spark SQL]({{ book.spark_sql }}/DataSourceRegister)) and registers itself as the *console* data source format. 6 | 7 | `ConsoleSinkProvider` is a `CreatableRelationProvider` ([Spark SQL]({{ book.spark_sql }}/CreatableRelationProvider)). 8 | 9 | ## Demo 10 | 11 | ```scala 12 | import org.apache.spark.sql.streaming.Trigger 13 | val q = spark 14 | .readStream 15 | .format("rate") 16 | .load 17 | .writeStream 18 | .format("console") // <-- requests ConsoleSinkProvider for a sink 19 | .trigger(Trigger.Once) 20 | .start 21 | ``` 22 | 23 | ```text 24 | scala> println(q.lastProgress.sink) 25 | { 26 | "description" : "org.apache.spark.sql.execution.streaming.ConsoleSinkProvider@2392cfb1" 27 | } 28 | ``` 29 | -------------------------------------------------------------------------------- /docs/datasources/console/ConsoleWriter.md: -------------------------------------------------------------------------------- 1 | # ConsoleWriter 2 | 3 | `ConsoleWriter` is a writer for [console](index.md) data source. 4 | -------------------------------------------------------------------------------- /docs/datasources/console/index.md: -------------------------------------------------------------------------------- 1 | # Console Data Source 2 | -------------------------------------------------------------------------------- /docs/datasources/file/FileStreamSourceCleaner.md: -------------------------------------------------------------------------------- 1 | # FileStreamSourceCleaner 2 | 3 | `FileStreamSourceCleaner` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/file/SinkFileStatus.md: -------------------------------------------------------------------------------- 1 | # SinkFileStatus 2 | 3 | [[creating-instance]] 4 | `SinkFileStatus` represents the status of files of [FileStreamSink](FileStreamSink.md) (and the type of the metadata of [FileStreamSinkLog](FileStreamSinkLog.md)): 5 | 6 | * [[path]] Path 7 | * [[size]] Size 8 | * [[isDir]] `isDir` flag 9 | * [[modificationTime]] Modification time 10 | * [[blockReplication]] Block replication 11 | * [[blockSize]] Block size 12 | * [[action]] Action (either [add](FileStreamSinkLog.md#ADD_ACTION) or [delete](FileStreamSinkLog.md#DELETE_ACTION)) 13 | 14 | === [[toFileStatus]] `toFileStatus` Method 15 | 16 | [source, scala] 17 | ---- 18 | toFileStatus: FileStatus 19 | ---- 20 | 21 | `toFileStatus` simply creates a new Hadoop [FileStatus]({{ hadoop.api }}/org/apache/hadoop/fs/FileStatus.html). 22 | 23 | NOTE: `toFileStatus` is used exclusively when `MetadataLogFileIndex` is [created](MetadataLogFileIndex.md). 24 | 25 | === [[apply]] Creating SinkFileStatus Instance 26 | 27 | [source, scala] 28 | ---- 29 | apply(f: FileStatus): SinkFileStatus 30 | ---- 31 | 32 | `apply` simply creates a new <> (with [add](FileStreamSinkLog.md#ADD_ACTION) action). 33 | 34 | `apply` is used when `ManifestFileCommitProtocol` is requested to [commitTask](ManifestFileCommitProtocol.md#commitTask). 35 | -------------------------------------------------------------------------------- /docs/datasources/file/index.md: -------------------------------------------------------------------------------- 1 | # File Data Source 2 | 3 | **File Data Source** comes with the following main abstractions: 4 | 5 | * [FileStreamSource](FileStreamSource.md) 6 | * [FileStreamSink](FileStreamSink.md) 7 | 8 | [FileStreamSink](FileStreamSink.md) uses [FileStreamSinkLog](FileStreamSinkLog.md) for [tracking valid files per micro-batch](FileStreamSink.md#addBatch) (as part of [ManifestFileCommitProtocol](ManifestFileCommitProtocol.md)). 9 | -------------------------------------------------------------------------------- /docs/datasources/foreach/ForeachSink.md: -------------------------------------------------------------------------------- 1 | # ForeachSink 2 | 3 | `ForeachSink` is a typed [streaming sink](../../Sink.md) that passes rows (of the type `T`) to [ForeachWriter](ForeachWriter.md) (one record at a time per partition). 4 | 5 | !!! note 6 | `ForeachSink` is assigned a `ForeachWriter` when `DataStreamWriter` is [started](../../DataStreamWriter.md#start). 7 | 8 | `ForeachSink` is used exclusively in [foreach](../../DataStreamWriter.md#foreach) operator. 9 | 10 | ```text 11 | val records = spark. 12 | readStream 13 | format("text"). 14 | load("server-logs/*.out"). 15 | as[String] 16 | 17 | import org.apache.spark.sql.ForeachWriter 18 | val writer = new ForeachWriter[String] { 19 | override def open(partitionId: Long, version: Long) = true 20 | override def process(value: String) = println(value) 21 | override def close(errorOrNull: Throwable) = {} 22 | } 23 | 24 | records.writeStream 25 | .queryName("server-logs processor") 26 | .foreach(writer) 27 | .start 28 | ``` 29 | 30 | Internally, `addBatch` (the only method from the <>) takes records from the input spark-sql-dataframe.md[DataFrame] (as `data`), transforms them to expected type `T` (of this `ForeachSink`) and (now as a spark-sql-dataset.md[Dataset]) spark-sql-dataset.md#foreachPartition[processes each partition]. 31 | 32 | [source, scala] 33 | ---- 34 | addBatch(batchId: Long, data: DataFrame): Unit 35 | ---- 36 | 37 | `addBatch` then opens the constructor's datasources/foreach/ForeachWriter.md[ForeachWriter] (for the spark-taskscheduler-taskcontext.md#getPartitionId[current partition] and the input batch) and passes the records to process (one at a time per partition). 38 | 39 | CAUTION: FIXME Why does Spark track whether the writer failed or not? Why couldn't it `finally` and do `close`? 40 | 41 | CAUTION: FIXME Can we have a constant for `"foreach"` for `source` in `DataStreamWriter`? 42 | -------------------------------------------------------------------------------- /docs/datasources/foreach/ForeachWriter.md: -------------------------------------------------------------------------------- 1 | # ForeachWriter 2 | 3 | `ForeachWriter` is the <> for a *foreach writer* that is a [streaming format](../../DataStreamWriter.md#foreach) that controls streaming writes. 4 | 5 | !!! note 6 | `ForeachWriter` is set using [DataStreamWriter.foreach](../../DataStreamWriter.md#foreach) operator. 7 | 8 | ```scala 9 | val foreachWriter = new ForeachWriter[String] { ... } 10 | streamingQuery. 11 | writeStream. 12 | foreach(foreachWriter). 13 | start 14 | ``` 15 | 16 | === [[contract]] ForeachWriter Contract 17 | 18 | [source, scala] 19 | ---- 20 | package org.apache.spark.sql 21 | 22 | abstract class ForeachWriter[T] { 23 | def open(partitionId: Long, version: Long): Boolean 24 | def process(value: T): Unit 25 | def close(errorOrNull: Throwable): Unit 26 | } 27 | ---- 28 | 29 | .ForeachWriter Contract 30 | [cols="1,2",options="header",width="100%"] 31 | |=== 32 | | Method 33 | | Description 34 | 35 | | [[open]] `open` 36 | | Used when... 37 | 38 | | [[process]] `process` 39 | | Used when... 40 | 41 | | [[close]] `close` 42 | | Used when... 43 | |=== 44 | -------------------------------------------------------------------------------- /docs/datasources/foreach/ForeachWriterProvider.md: -------------------------------------------------------------------------------- 1 | == [[ForeachWriterProvider]] ForeachWriterProvider 2 | 3 | `ForeachWriterProvider` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/foreach/ForeachWriterTable.md: -------------------------------------------------------------------------------- 1 | # ForeachWriterTable 2 | 3 | `ForeachWriterTable` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/foreach/index.md: -------------------------------------------------------------------------------- 1 | # Foreach Data Source 2 | -------------------------------------------------------------------------------- /docs/datasources/index.md: -------------------------------------------------------------------------------- 1 | # Data Sources 2 | 3 | Spark Structured Streaming comes with a variety of built-in streaming data sources. 4 | -------------------------------------------------------------------------------- /docs/datasources/memory/ContinuousMemoryStream.md: -------------------------------------------------------------------------------- 1 | == [[ContinuousMemoryStream]] ContinuousMemoryStream 2 | 3 | `ContinuousMemoryStream` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/memory/MemoryPlan.md: -------------------------------------------------------------------------------- 1 | # MemoryPlan Logical Operator 2 | 3 | `MemoryPlan` is a leaf logical operator (i.e. `LogicalPlan`) that is used to query the data that has been written into a [MemorySink](MemorySink.md). `MemoryPlan` is created when [starting continuous writing](../../DataStreamWriter.md#start) (to a `MemorySink`). 4 | 5 | TIP: See the example in [MemoryStream](MemoryStream.md). 6 | 7 | ```text 8 | scala> intsOut.explain(true) 9 | == Parsed Logical Plan == 10 | SubqueryAlias memstream 11 | +- MemoryPlan org.apache.spark.sql.execution.streaming.MemorySink@481bf251, [value#21] 12 | 13 | == Analyzed Logical Plan == 14 | value: int 15 | SubqueryAlias memstream 16 | +- MemoryPlan org.apache.spark.sql.execution.streaming.MemorySink@481bf251, [value#21] 17 | 18 | == Optimized Logical Plan == 19 | MemoryPlan org.apache.spark.sql.execution.streaming.MemorySink@481bf251, [value#21] 20 | 21 | == Physical Plan == 22 | LocalTableScan [value#21] 23 | ``` 24 | 25 | When executed, `MemoryPlan` is translated to `LocalTableScanExec` physical operator (similar to `LocalRelation` logical operator) in `BasicOperators` execution planning strategy. 26 | -------------------------------------------------------------------------------- /docs/datasources/memory/MemoryStreamBase.md: -------------------------------------------------------------------------------- 1 | # MemoryStreamBase 2 | 3 | `MemoryStreamBase` is an [extension](#contract) of the [SparkDataStream](../../SparkDataStream.md) abstraction for [data streams](#implementations) that [keep data](#addData) in memory. 4 | 5 | ## Contract 6 | 7 | ### Adding Data 8 | 9 | ```scala 10 | addData( 11 | data: A*): Offset 12 | addData( 13 | data: TraversableOnce[A]): Offset 14 | ``` 15 | 16 | Adds new data to this memory stream (and advances the current [Offset](../../Offset.md)) 17 | 18 | See [MemoryStream](MemoryStream.md#addData) 19 | 20 | ## Implementations 21 | 22 | * [ContinuousMemoryStream](ContinuousMemoryStream.md) 23 | * [MemoryStream](MemoryStream.md) 24 | 25 | ## Creating Instance 26 | 27 | `MemoryStreamBase` takes the following to be created: 28 | 29 | * `SQLContext` ([Spark SQL]({{ book.spark_sql }}/SQLContext)) 30 | 31 | !!! note "Abstract Class" 32 | `MemoryStreamBase` is an abstract class and cannot be created directly. It is created indirectly for the [concrete MemoryStreamBases](#implementations). 33 | -------------------------------------------------------------------------------- /docs/datasources/memory/MemoryStreamingWrite.md: -------------------------------------------------------------------------------- 1 | # MemoryStreamingWrite 2 | 3 | `MemoryStreamingWrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/rate-micro-batch/RatePerMicroBatchProvider.md: -------------------------------------------------------------------------------- 1 | # RatePerMicroBatchProvider 2 | 3 | `RatePerMicroBatchProvider` is a `SimpleTableProvider` ([Spark SQL]({{ book.spark_sql }}/connector/SimpleTableProvider)). 4 | 5 | ## DataSourceRegister 6 | 7 | `RatePerMicroBatchProvider` is a `DataSourceRegister` ([Spark SQL]({{ book.spark_sql }}/DataSourceRegister)) that registers `rate-micro-batch` alias. 8 | 9 | ## Creating Table 10 | 11 | ```scala 12 | getTable( 13 | options: CaseInsensitiveStringMap): Table 14 | ``` 15 | 16 | `getTable` creates a [RatePerMicroBatchTable](RatePerMicroBatchTable.md) with the [options](options.md) (given the `CaseInsensitiveStringMap`). 17 | 18 | --- 19 | 20 | `getTable` is part of the `SimpleTableProvider` ([Spark SQL]({{ book.spark_sql }}/connector/SimpleTableProvider#getTable)) abstraction. 21 | -------------------------------------------------------------------------------- /docs/datasources/rate-micro-batch/RatePerMicroBatchTable.md: -------------------------------------------------------------------------------- 1 | # RatePerMicroBatchTable 2 | 3 | `RatePerMicroBatchTable` is a `Table` ([Spark SQL]({{ book.spark_sql }}/connector/Table)) that `SupportsRead` ([Spark SQL]({{ book.spark_sql }}/connector/SupportsRead)). 4 | 5 | ## Creating Instance 6 | 7 | `RatePerMicroBatchTable` takes the following to be created: 8 | 9 | * [rowsPerBatch](options.md#rowsPerBatch) 10 | * [numPartitions](options.md#numPartitions) 11 | * [startTimestamp](options.md#startTimestamp) 12 | * [advanceMillisPerBatch](options.md#advanceMillisPerBatch) 13 | 14 | `RatePerMicroBatchTable` is created when: 15 | 16 | * `RatePerMicroBatchProvider` is requested for the [table](RatePerMicroBatchProvider.md#getTable) 17 | 18 | ## schema 19 | 20 | ```scala 21 | schema(): StructType 22 | ``` 23 | 24 | Name | Data Type 25 | -----|---------- 26 | timestamp | TimestampType 27 | value | LongType 28 | 29 | `schema` is part of the `Table` ([Spark SQL]({{ book.spark_sql }}/connector/Table#schema)) abstraction. 30 | 31 | ## Creating ScanBuilder 32 | 33 | ```scala 34 | newScanBuilder( 35 | options: CaseInsensitiveStringMap): ScanBuilder 36 | ``` 37 | 38 | `newScanBuilder` is part of the `SupportsRead` ([Spark SQL]({{ book.spark_sql }}/connector/SupportsRead#newScanBuilder)) abstraction. 39 | 40 | --- 41 | 42 | `newScanBuilder` creates a new `Scan` ([Spark SQL]({{ book.spark_sql }}/connector/Scan)) that creates a [RatePerMicroBatchStream](RatePerMicroBatchStream.md) when requested for a `MicroBatchStream` ([Spark SQL]({{ book.spark_sql }}/connector/Scan#toMicroBatchStream)). 43 | -------------------------------------------------------------------------------- /docs/datasources/rate-micro-batch/index.md: -------------------------------------------------------------------------------- 1 | # Rate Per Micro-Batch Data Source 2 | 3 | **Rate Per Micro-Batch Data Source** provides a consistent number of rows per microbatch. 4 | 5 | From [this commit]({{ spark.commit }}/70fde44e930926cbcd1fc95fa7cfb915c25cff9c): 6 | 7 | > This proposes to introduce a new data source having short name as "rate-micro-batch", which produces similar input rows as "rate" (increment long values with timestamps), but ensures that each micro-batch has a "predictable" set of input rows. 8 | 9 | > "rate-micro-batch" data source receives a config to specify the number of rows per micro-batch, which defines the set of input rows for further micro-batches. For example, if the number of rows per micro-batch is set to 1000, the first batch would have 1000 rows having value range as `0~999`, the second batch would have 1000 rows having value range as `1000~1999`, and so on. This characteristic brings different use cases compared to rate data source, as we can't predict the input rows for rate data source like this. 10 | 11 | > For generated time (timestamp column), the data source applies the same mechanism to make the value of column be predictable. `startTimestamp` option defines the starting value of generated time, and `advanceMillisPerBatch` option defines how much time the generated time should advance per micro-batch. All input rows in the same micro-batch will have same timestamp. 12 | 13 | Rate Per Micro-Batch data source is a new feature of Apache Spark 3.3.0 ([SPARK-37062]({{ spark.jira }}/SPARK-37062)). 14 | 15 | ## Internals 16 | 17 | Rate Per Micro-Batch Data Source is registered by [RatePerMicroBatchProvider](RatePerMicroBatchProvider.md) to be available under [rate-micro-batch](RatePerMicroBatchProvider.md#rate-micro-batch) alias. 18 | 19 | `RatePerMicroBatchProvider` uses [RatePerMicroBatchTable](RatePerMicroBatchTable.md) as the [Table](RatePerMicroBatchProvider.md#getTable) ([Spark SQL]({{ book.spark_sql }}/connector/Table/)). 20 | 21 | When requested for a [MicroBatchStream](RatePerMicroBatchTable.md#newScanBuilder), `RatePerMicroBatchTable` creates a [RatePerMicroBatchStream](RatePerMicroBatchStream.md) with extra support for [Trigger.AvailableNow](../../Trigger.md#AvailableNow) mode. 22 | 23 | Rate Per Micro-Batch Data Source supports [options](options.md) (esp. [rowsPerBatch](options.md#rowsPerBatch) and [advanceMillisPerBatch](options.md#advanceMillisPerBatch) for [Trigger.AvailableNow](RatePerMicroBatchStream.md#latestOffset) mode). 24 | -------------------------------------------------------------------------------- /docs/datasources/rate-micro-batch/options.md: -------------------------------------------------------------------------------- 1 | # Options 2 | 3 | ## advanceMillisPerBatch 4 | 5 | default: `1000` 6 | 7 | ## numPartitions 8 | 9 | default: `SparkSession.active.sparkContext.defaultParallelism` 10 | 11 | ## rowsPerBatch 12 | 13 | default: `0` 14 | 15 | ## startTimestamp 16 | 17 | default: `0` 18 | -------------------------------------------------------------------------------- /docs/datasources/rate/RateStreamContinuousReader.md: -------------------------------------------------------------------------------- 1 | # RateStreamContinuousReader 2 | 3 | `RateStreamContinuousReader` is a [ContinuousReader](../../continuous-execution/ContinuousReader.md) that...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/rate/RateStreamProvider.md: -------------------------------------------------------------------------------- 1 | # RateStreamProvider 2 | 3 | `RateStreamProvider` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/datasources/rate/index.md: -------------------------------------------------------------------------------- 1 | # Rate Data Source 2 | -------------------------------------------------------------------------------- /docs/datasources/socket/TextSocketSourceProvider.md: -------------------------------------------------------------------------------- 1 | # TextSocketSourceProvider 2 | 3 | `TextSocketSourceProvider` is a [StreamSourceProvider](../../StreamSourceProvider.md) for [Text Socket Data Source](index.md). 4 | 5 | `TextSocketSourceProvider` requires two options (that you can set using `option` method): 6 | 7 | 1. `host` which is the host name. 8 | 2. `port` which is the port number. It must be an integer. 9 | 10 | `TextSocketSourceProvider` also supports <> option that is a boolean flag that you can use to include timestamps in the schema. 11 | 12 | ## DataSourceRegister 13 | 14 | `TextSocketSourceProvider` is a `DataSourceRegister` with the short name of **socket**. 15 | 16 | ## includeTimestamp Option 17 | 18 | ## createSource 19 | 20 | `createSource` creates a [TextSocketSource](TextSocketSource.md) (with the `host` and `port`). 21 | 22 | ## sourceSchema 23 | 24 | `sourceSchema` returns `textSocket` as the name of the source and the schema that can be one of the two available schemas: 25 | 26 | 1. `SCHEMA_REGULAR` (default) which is a schema with a single `value` field of String type. 27 | 28 | 2. `SCHEMA_TIMESTAMP` when `<>` flag option is set. It is not, i.e. `false`, by default. The schema are `value` field of `StringType` type and `timestamp` field of spark-sql-DataType.md#TimestampType[TimestampType] type of format `yyyy-MM-dd HH:mm:ss`. 29 | 30 | TIP: Read about spark-sql-schema.md[schema]. 31 | 32 | Internally, it starts by printing out the following WARN message to the logs: 33 | 34 | ```text 35 | The socket source should not be used for production applications! It does not support recovery and stores state indefinitely. 36 | ``` 37 | 38 | It then checks whether `host` and `port` parameters are defined and if not it throws a `AnalysisException`: 39 | 40 | ```text 41 | Set a host to read from with option("host", ...). 42 | ``` 43 | -------------------------------------------------------------------------------- /docs/datasources/socket/index.md: -------------------------------------------------------------------------------- 1 | # Text Socket Data Source 2 | 3 | **Text Socket Data Source** comes with the following main abstractions: 4 | 5 | * [TextSocketSourceProvider](TextSocketSourceProvider.md) 6 | * [TextSocketSource](TextSocketSource.md) 7 | -------------------------------------------------------------------------------- /docs/demo/StateStoreSaveExec-Update.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: StateStoreSaveExec with Update Output Mode 7 | 8 | CAUTION: FIXME Example of Update with StateStoreSaveExec (and optional watermark) 9 | -------------------------------------------------------------------------------- /docs/demo/groupByKey-count-Update.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: groupByKey Streaming Aggregation in Update Mode 7 | 8 | The example shows [Dataset.groupByKey](../operators/groupByKey.md) streaming operator to count rows in [Update](../OutputMode.md#Update) output mode. 9 | 10 | In other words, it is an example of using `Dataset.groupByKey` with `count` aggregation function to count customer orders (`T`) per zip code (`K`). 11 | 12 | ```text 13 | package pl.japila.spark.examples 14 | 15 | import org.apache.spark.sql.SparkSession 16 | import org.apache.spark.sql.streaming.{OutputMode, Trigger} 17 | 18 | object GroupByKeyStreamingApp extends App { 19 | 20 | val inputTopic = "GroupByKeyApp-input" 21 | val appName = this.getClass.getSimpleName.replace("$", "") 22 | 23 | val spark = SparkSession.builder 24 | .master("local[*]") 25 | .appName(appName) 26 | .getOrCreate 27 | import spark.implicits._ 28 | 29 | case class Order(id: Long, zipCode: String) 30 | 31 | // Input (source node) 32 | val orders = spark 33 | .readStream 34 | .format("kafka") 35 | .option("startingOffsets", "latest") 36 | .option("subscribe", inputTopic) 37 | .option("kafka.bootstrap.servers", ":9092") 38 | .load 39 | .select($"offset" as "id", $"value" as "zipCode") // FIXME Use csv, json, avro 40 | .as[Order] 41 | 42 | // Processing logic 43 | // groupByKey + count 44 | val byZipCode = (o: Order) => o.zipCode 45 | val ordersByZipCode = orders.groupByKey(byZipCode) 46 | 47 | import org.apache.spark.sql.functions.count 48 | val typedCountCol = (count("zipCode") as "count").as[String] 49 | val counts = ordersByZipCode 50 | .agg(typedCountCol) 51 | .select($"value" as "zip_code", $"count") 52 | 53 | // Output (sink node) 54 | import scala.concurrent.duration._ 55 | counts 56 | .writeStream 57 | .format("console") 58 | .outputMode(OutputMode.Update) // FIXME Use Complete 59 | .queryName(appName) 60 | .trigger(Trigger.ProcessingTime(5.seconds)) 61 | .start 62 | .awaitTermination() 63 | } 64 | ``` 65 | 66 | ## Credits 67 | 68 | * The example with customer orders and postal codes is borrowed from Apache Beam's [Using GroupByKey](https://beam.apache.org/documentation/programming-guide/#transforms-gbk) Programming Guide. 69 | -------------------------------------------------------------------------------- /docs/demo/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | - navigation 5 | --- 6 | 7 | # Demo 8 | 9 | * [Arbitrary Stateful Streaming Aggregation with KeyValueGroupedDataset.flatMapGroupsWithState Operator](arbitrary-stateful-streaming-aggregation-flatMapGroupsWithState.md) 10 | * [current_timestamp Function For Processing Time in Streaming Queries](current_timestamp.md) 11 | * [Custom Streaming Sink (and Monitoring SQL Queries in web UI)](custom-sink-webui.md) 12 | * [Deep Dive into FileStreamSink](deep-dive-into-filestreamsink.md) 13 | * [Exploring Checkpointed State](exploring-checkpointed-state.md) 14 | * [groupByKey Streaming Aggregation in Update Mode](groupByKey-count-Update.md) 15 | * [Internals of FlatMapGroupsWithStateExec Physical Operator](spark-sql-streaming-demo-FlatMapGroupsWithStateExec.md) 16 | * [Kafka Data Source](kafka-data-source.md) 17 | * [Streaming Windowed Aggregation](streaming-windowed-aggregation.md) 18 | * [RocksDB State Store for Streaming Aggregation](rocksdb-state-store-for-streaming-aggregation.md) 19 | * [Stream-Stream Inner Join](stream-stream-inner-join.md) 20 | * [Streaming Query for Running Counts (Socket Source and Complete Output Mode)](groupBy-running-count-complete.md) 21 | * [Streaming Watermark](streaming-watermark.md) 22 | * [Streaming Watermark with Aggregation in Append Output Mode](watermark-aggregation-append.md) 23 | * [StateStoreSaveExec with Complete Output Mode](StateStoreSaveExec-Complete.md) 24 | * [StateStoreSaveExec with Update Output Mode](StateStoreSaveExec-Update.md) 25 | * [Using StreamingQueryManager for Query Termination Management](StreamingQueryManager-awaitAnyTermination-resetTerminated.md) 26 | * [Using File Streaming Source](using-file-streaming-source.md) 27 | -------------------------------------------------------------------------------- /docs/demo/stream-stream-inner-join.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: Stream-Stream Inner Join 7 | 8 | This demo shows a streaming query with a [stream-stream inner join](../join/index.md#stream-stream-joins). 9 | 10 | The source code of this demo is in the [spark-examples](https://github.com/jaceklaskowski/spark-examples/blob/main/src/main/scala/pl/japila/spark/sql/streaming/StreamStreamJoinDemo.scala) repository. 11 | -------------------------------------------------------------------------------- /docs/demo/streaming-watermark.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: Streaming Watermark 7 | 8 | This demo shows the internals of [streaming watermark](../watermark/index.md) with [Kafka Data Source](../kafka/index.md). 9 | 10 | !!! note 11 | Please start a Kafka cluster and `spark-shell` as described in [Demo: Kafka Data Source](kafka-data-source.md). 12 | 13 | ## Streaming Query with Watermark 14 | 15 | ```scala 16 | import java.time.Clock 17 | val timeOffset = Clock.systemUTC.instant.getEpochSecond 18 | ``` 19 | 20 | ```scala 21 | val queryName = s"Demo: Streaming Watermark ($timeOffset)" 22 | val checkpointLocation = s"/tmp/demo-checkpoint-$timeOffset" 23 | ``` 24 | 25 | ```scala 26 | import org.apache.spark.sql.streaming.Trigger 27 | import scala.concurrent.duration._ 28 | val eventTimeCol = (($"tokens"(2) cast "long") + lit(timeOffset)) cast "timestamp" as "event_time" 29 | val sq = spark 30 | .readStream 31 | .format("kafka") 32 | .option("subscribe", "demo.streaming-watermark") 33 | .option("kafka.bootstrap.servers", ":9092") 34 | .load 35 | .select($"value" cast "string") 36 | .select(split($"value", ",") as "tokens") 37 | .select( 38 | $"tokens"(0) as "id" cast "long", 39 | $"tokens"(1) as "name", 40 | eventTimeCol) 41 | .withWatermark(eventTime = "event_time", delayThreshold = "5 seconds") 42 | .writeStream 43 | .format("console") 44 | .queryName(queryName) 45 | .trigger(Trigger.ProcessingTime(1.seconds)) 46 | .option("checkpointLocation", checkpointLocation) 47 | .option("truncate", false) 48 | .start 49 | ``` 50 | 51 | ## Send Events 52 | 53 | ```console 54 | echo "0,zero,0" | kcat -P -b :9092 -t demo.streaming-watermark 55 | ``` 56 | 57 | ```console 58 | echo "10,ten,10" | kcat -P -b :9092 -t demo.streaming-watermark 59 | ``` 60 | -------------------------------------------------------------------------------- /docs/demo/using-file-streaming-source.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: Using File Streaming Source 7 | 8 | This demo shows a streaming query that reads files using [FileStreamSource](../datasources/file/FileStreamSource.md). 9 | 10 | ## Prerequisites 11 | 12 | Make sure that the source directory is available before starting the query. 13 | 14 | ```shell 15 | mkdir /tmp/text-logs 16 | ``` 17 | 18 | ## Configure Logging 19 | 20 | Enable [logging](../datasources/file/FileStreamSource.md#logging) for `FileStreamSource`. 21 | 22 | ## Start Streaming Query 23 | 24 | Use `spark-shell` for fast interactive prototyping. 25 | 26 | Describe a source to load data from. 27 | 28 | ```scala 29 | val lines = spark 30 | .readStream 31 | .format("text") 32 | .option("maxFilesPerTrigger", 1) 33 | .load("/tmp/text-logs") 34 | ``` 35 | 36 | Show the schema. 37 | 38 | ```text 39 | scala> lines.printSchema 40 | root 41 | |-- value: string (nullable = true) 42 | ``` 43 | 44 | Describe the sink (`console`) and start the streaming query. 45 | 46 | ```scala 47 | import org.apache.spark.sql.streaming.Trigger 48 | import concurrent.duration._ 49 | val interval = 15.seconds 50 | val trigger = Trigger.ProcessingTime(interval) 51 | val queryName = s"one file every micro-batch (every $interval)" 52 | val sq = lines 53 | .writeStream 54 | .format("console") 55 | .option("checkpointLocation", "/tmp/checkpointLocation") 56 | .trigger(trigger) 57 | .queryName(queryName) 58 | .start 59 | ``` 60 | 61 | Use web UI to monitor the query (http://localhost:4040). 62 | 63 | ## Stop Query 64 | 65 | ```scala 66 | spark.streams.active.foreach(_.stop) 67 | ``` 68 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/StreamingDeduplicationStrategy.md: -------------------------------------------------------------------------------- 1 | # StreamingDeduplicationStrategy Execution Planning Strategy 2 | 3 | `StreamingDeduplicationStrategy` is an execution planning strategy that can plan streaming queries with `Deduplicate` logical operators (over streaming queries) to [StreamingDeduplicateExec](../physical-operators/StreamingDeduplicateExec.md) physical operators. 4 | 5 | `StreamingDeduplicationStrategy` is available using `SessionState`. 6 | 7 | ```scala 8 | spark.sessionState.planner.StreamingDeduplicationStrategy 9 | ``` 10 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/StreamingGlobalLimitStrategy.md: -------------------------------------------------------------------------------- 1 | # StreamingGlobalLimitStrategy Execution Planning Strategy 2 | 3 | `StreamingGlobalLimitStrategy` is an execution planning strategy that can plan streaming queries with `ReturnAnswer` and `Limit` logical operators (over streaming queries) with the [Append](#outputMode) output mode to [StreamingGlobalLimitExec](../physical-operators/StreamingGlobalLimitExec.md) physical operator. 4 | 5 | `StreamingGlobalLimitStrategy` is used (and created) when [IncrementalExecution](../IncrementalExecution.md) is requested to plan a streaming query. 6 | 7 | ## Creating Instance 8 | 9 | `StreamingGlobalLimitStrategy` takes a single [OutputMode](../OutputMode.md) to be created (which is the [OutputMode](../IncrementalExecution.md#outputMode) of the [IncrementalExecution](../IncrementalExecution.md)). 10 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/StreamingJoinStrategy.md: -------------------------------------------------------------------------------- 1 | # StreamingJoinStrategy Execution Planning Strategy — Stream-Stream Equi-Joins 2 | 3 | `StreamingJoinStrategy` is an execution planning strategy that can plan streaming queries with `Join` logical operators of two streaming queries to a [StreamingSymmetricHashJoinExec](../physical-operators/StreamingSymmetricHashJoinExec.md) physical operator. 4 | 5 | `StreamingJoinStrategy` throws an `AnalysisException` when applied to a `Join` logical operator with no equality predicate: 6 | 7 | ```text 8 | Stream-stream join without equality predicate is not supported 9 | ``` 10 | 11 | `StreamingJoinStrategy` is used when [IncrementalExecution](../IncrementalExecution.md) is requested to plan a streaming query. 12 | 13 | [[logging]] 14 | [TIP] 15 | ==== 16 | `StreamingJoinStrategy` does not print out any messages to the logs. `StreamingJoinStrategy` however uses `ExtractEquiJoinKeys` ([Spark SQL]({{ book.spark_sql }}/ExtractEquiJoinKeys)) Scala extractor for destructuring `Join` logical operators that does print out DEBUG messages to the logs. 17 | 18 | Enable `ALL` logging level for `org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys` to see what happens inside. 19 | 20 | Add the following line to `conf/log4j.properties`: 21 | 22 | ```text 23 | log4j.logger.org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys=ALL 24 | ``` 25 | 26 | Refer to <>. 27 | ==== 28 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/StreamingRelationStrategy.md: -------------------------------------------------------------------------------- 1 | # StreamingRelationStrategy Execution Planning Strategy 2 | 3 | `StreamingRelationStrategy` is an execution planning strategy that can plan streaming queries with [StreamingRelation](../logical-operators/StreamingRelation.md), [StreamingExecutionRelation](../logical-operators/StreamingExecutionRelation.md), and [StreamingRelationV2](../logical-operators/StreamingRelationV2.md) logical operators to [StreamingRelationExec](../physical-operators/StreamingRelationExec.md) physical operators. 4 | 5 | ![StreamingRelationStrategy, StreamingRelation, StreamingExecutionRelation and StreamingRelationExec Operators](../images/StreamingRelationStrategy-apply.png) 6 | 7 | `StreamingRelationStrategy` is used when [IncrementalExecution](../IncrementalExecution.md) is requested to plan a streaming query. 8 | 9 | ## Accessing StreamingRelationStrategy 10 | 11 | `StreamingRelationStrategy` is available using `SessionState` (of a `SparkSession`). 12 | 13 | ```scala 14 | spark.sessionState.planner.StreamingRelationStrategy 15 | ``` 16 | 17 | ## Demo 18 | 19 | ```text 20 | val rates = spark. 21 | readStream. 22 | format("rate"). 23 | load // <-- gives a streaming Dataset with a logical plan with StreamingRelation logical operator 24 | 25 | // StreamingRelation logical operator for the rate streaming source 26 | scala> println(rates.queryExecution.logical.numberedTreeString) 27 | 00 StreamingRelation DataSource(org.apache.spark.sql.SparkSession@31ba0af0,rate,List(),None,List(),None,Map(),None), rate, [timestamp#0, value#1L] 28 | 29 | // StreamingRelationExec physical operator (shown without "Exec" suffix) 30 | scala> rates.explain 31 | == Physical Plan == 32 | StreamingRelation rate, [timestamp#0, value#1L] 33 | 34 | // Let's do the planning manually 35 | import spark.sessionState.planner.StreamingRelationStrategy 36 | val physicalPlan = StreamingRelationStrategy.apply(rates.queryExecution.logical).head 37 | scala> println(physicalPlan.numberedTreeString) 38 | 00 StreamingRelation rate, [timestamp#0, value#1L] 39 | ``` 40 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/index.md: -------------------------------------------------------------------------------- 1 | # Execution Planning Strategies 2 | -------------------------------------------------------------------------------- /docs/features/index.md: -------------------------------------------------------------------------------- 1 | # Features 2 | -------------------------------------------------------------------------------- /docs/images/ContinuousExecution-addOffset.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/ContinuousExecution-addOffset.png -------------------------------------------------------------------------------- /docs/images/DataStreamReader-SparkSession-StreamingRelation.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/DataStreamReader-SparkSession-StreamingRelation.png -------------------------------------------------------------------------------- /docs/images/FlatMapGroupsWithStateExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/FlatMapGroupsWithStateExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/KafkaOffsetReader-fetchSpecificOffsets.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/KafkaOffsetReader-fetchSpecificOffsets.png -------------------------------------------------------------------------------- /docs/images/KafkaSource-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/KafkaSource-creating-instance.png -------------------------------------------------------------------------------- /docs/images/KafkaSource-initialPartitionOffsets.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/KafkaSource-initialPartitionOffsets.png -------------------------------------------------------------------------------- /docs/images/KeyToNumValuesStore-KeyWithIndexToValueStore.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/KeyToNumValuesStore-KeyWithIndexToValueStore.png -------------------------------------------------------------------------------- /docs/images/MicroBatchExecution-constructNextBatch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/MicroBatchExecution-constructNextBatch.png -------------------------------------------------------------------------------- /docs/images/MicroBatchExecution.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/MicroBatchExecution.png -------------------------------------------------------------------------------- /docs/images/OneSideHashJoiner.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/OneSideHashJoiner.png -------------------------------------------------------------------------------- /docs/images/ProgressReporter-currentDurationsMs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/ProgressReporter-currentDurationsMs.png -------------------------------------------------------------------------------- /docs/images/ProgressReporter-finishTrigger-timestamps.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/ProgressReporter-finishTrigger-timestamps.png -------------------------------------------------------------------------------- /docs/images/ProgressReporter-updateProgress.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/ProgressReporter-updateProgress.png -------------------------------------------------------------------------------- /docs/images/StateStoreRDD-SparkPlans-LogicalPlans-operators.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreRDD-SparkPlans-LogicalPlans-operators.png -------------------------------------------------------------------------------- /docs/images/StateStoreRDD-StateStoreCoordinator.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreRDD-StateStoreCoordinator.png -------------------------------------------------------------------------------- /docs/images/StateStoreRestoreExec-IncrementalExecution.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreRestoreExec-IncrementalExecution.png -------------------------------------------------------------------------------- /docs/images/StateStoreRestoreExec-StatefulAggregationStrategy.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreRestoreExec-StatefulAggregationStrategy.png -------------------------------------------------------------------------------- /docs/images/StateStoreRestoreExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreRestoreExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/StateStoreSaveExec-IncrementalExecution.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreSaveExec-IncrementalExecution.png -------------------------------------------------------------------------------- /docs/images/StateStoreSaveExec-StateStoreRDD-count.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreSaveExec-StateStoreRDD-count.png -------------------------------------------------------------------------------- /docs/images/StateStoreSaveExec-StateStoreRDD.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreSaveExec-StateStoreRDD.png -------------------------------------------------------------------------------- /docs/images/StateStoreSaveExec-StatefulAggregationStrategy.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreSaveExec-StatefulAggregationStrategy.png -------------------------------------------------------------------------------- /docs/images/StateStoreSaveExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StateStoreSaveExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-creating-instance.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-getBatchDescriptionString-webUI.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-getBatchDescriptionString-webUI.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatch-addBatch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatch-addBatch.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatch-getBatch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatch-getBatch.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatch-newBatchesPlan.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatch-newBatchesPlan.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatch-nextBatch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatch-nextBatch.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatch-queryPlanning.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatch-queryPlanning.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-runBatches.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-runBatches.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-start.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-start.png -------------------------------------------------------------------------------- /docs/images/StreamExecution-uniqueSources.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamExecution-uniqueSources.png -------------------------------------------------------------------------------- /docs/images/StreamingDeduplicateExec-StreamingDeduplicationStrategy.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingDeduplicateExec-StreamingDeduplicationStrategy.png -------------------------------------------------------------------------------- /docs/images/StreamingDeduplicateExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingDeduplicateExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/StreamingExecutionRelation.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingExecutionRelation.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryListener-onQueryProgress.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryListener-onQueryProgress.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryListener-onQueryStarted.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryListener-onQueryStarted.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryListener-onQueryTerminated.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryListener-onQueryTerminated.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryListenerBus.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryListenerBus.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryManager-createQuery.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryManager-createQuery.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryManager-notifyQueryTermination.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryManager-notifyQueryTermination.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryManager-postListenerEvent.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryManager-postListenerEvent.png -------------------------------------------------------------------------------- /docs/images/StreamingQueryManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingQueryManager.png -------------------------------------------------------------------------------- /docs/images/StreamingRelation.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingRelation.png -------------------------------------------------------------------------------- /docs/images/StreamingRelationStrategy-apply.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingRelationStrategy-apply.png -------------------------------------------------------------------------------- /docs/images/StreamingSymmetricHashJoinExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/StreamingSymmetricHashJoinExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/SymmetricHashJoinStateManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/SymmetricHashJoinStateManager.png -------------------------------------------------------------------------------- /docs/images/memory-sink-webui-createviewcommand.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/memory-sink-webui-createviewcommand.png -------------------------------------------------------------------------------- /docs/images/webui-completed-streaming-queries.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-completed-streaming-queries.png -------------------------------------------------------------------------------- /docs/images/webui-spark-job-streaming-query-started.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-spark-job-streaming-query-started.png -------------------------------------------------------------------------------- /docs/images/webui-sql-completed-queries-one-per-batch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-sql-completed-queries-one-per-batch.png -------------------------------------------------------------------------------- /docs/images/webui-sql-completed-queries-three-per-batch.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-sql-completed-queries-three-per-batch.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-aggregated-number-of-rows-dropped-by-watermark.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-aggregated-number-of-rows-dropped-by-watermark.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-aggregated-number-of-total-state-rows.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-aggregated-number-of-total-state-rows.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-aggregated-number-of-updated-state-rows.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-aggregated-number-of-updated-state-rows.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-aggregated-state-memory-used-in-bytes.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-aggregated-state-memory-used-in-bytes.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-basic-info.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-basic-info.png -------------------------------------------------------------------------------- /docs/images/webui-statistics-global-watermark-gap.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/webui-statistics-global-watermark-gap.png -------------------------------------------------------------------------------- /docs/images/workshop-tweet.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/docs/images/workshop-tweet.png -------------------------------------------------------------------------------- /docs/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | - navigation 5 | --- 6 | 7 | # The Internals of {{ book.title }} (Apache Spark {{ spark.version }}) 8 | 9 | Welcome to **The Internals of {{ book.title }}** online book! 🤙 10 | 11 | I'm [Jacek Laskowski](https://pl.linkedin.com/in/jaceklaskowski), an IT freelancer specializing in [Apache Spark](https://books.japila.pl/apache-spark-internals/) (incl. [Spark SQL](https://books.japila.pl/spark-sql-internals/) and [Spark Structured Streaming](https://books.japila.pl/spark-structured-streaming-internals/)), [Delta Lake](https://books.japila.pl/delta-lake-internals/) and [Apache Kafka](https://books.japila.pl/kafka-internals/) (incl. [Kafka Streams](https://books.japila.pl/kafka-streams-internals/) and [ksqlDB](https://books.japila.pl/ksqldb-internals/)) (with brief forays into a wider data engineering space, e.g. [Trino](https://trino.io/), [Dask](https://www.dask.org/) and [dbt](https://www.getdbt.com/), mostly during [Warsaw Data Engineering](https://www.meetup.com/Warsaw-Data-Engineering/) meetups). 12 | 13 | I'm very excited to have you here and hope you will enjoy exploring the internals of {{ book.title }} as much as I have. 14 | 15 | !!! quote "Flannery O'Connor" 16 | I write to discover what I know. 17 | 18 | !!! note ""The Internals Of" series" 19 | I'm also writing other online books in the "The Internals Of" series. Please visit ["The Internals Of" Online Books](https://books.japila.pl) home page. 20 | 21 | Expect text and code snippets from a variety of public sources. Attribution follows. 22 | 23 | Now, let's take a deep dive into [{{ book.title }}](overview.md) 🔥 24 | 25 | --- 26 | 27 | Last update: {{ git.date.strftime('%Y-%m-%d') }} 28 | -------------------------------------------------------------------------------- /docs/join/.pages: -------------------------------------------------------------------------------- 1 | title: Streaming Join 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/join/JoinStateWatermarkPredicates.md: -------------------------------------------------------------------------------- 1 | # JoinStateWatermarkPredicates 2 | 3 | [[creating-instance]] 4 | `JoinStateWatermarkPredicates` contains watermark predicates for state removal of the children of a [StreamingSymmetricHashJoinExec](../physical-operators/StreamingSymmetricHashJoinExec.md) physical operator: 5 | 6 | * [[left]] <> for the left-hand side of a join (default: `None`) 7 | 8 | * [[right]] <> for the right-hand side of a join (default: `None`) 9 | 10 | `JoinStateWatermarkPredicates` is <> for the following: 11 | 12 | * <> physical operator is created (with the optional properties undefined, including <>) 13 | 14 | * `StreamingSymmetricHashJoinHelper` utility is requested for [one](StreamingSymmetricHashJoinHelper.md#getStateWatermarkPredicates) (for `IncrementalExecution` for the [state preparation rule](../IncrementalExecution.md#state) to optimize and specify the execution-specific configuration for a query plan with [StreamingSymmetricHashJoinExec](../physical-operators/StreamingSymmetricHashJoinExec.md) physical operators) 15 | 16 | === [[toString]] Textual Representation -- `toString` Method 17 | 18 | [source, scala] 19 | ---- 20 | toString: String 21 | ---- 22 | 23 | NOTE: `toString` is part of the ++https://docs.oracle.com/javase/8/docs/api/java/lang/Object.html#toString--++[java.lang.Object] contract for the string representation of the object. 24 | 25 | `toString` uses the <> and <> predicates for the string representation: 26 | 27 | ``` 28 | state cleanup [ left [left], right [right] ] 29 | ``` 30 | -------------------------------------------------------------------------------- /docs/join/StateStoreAwareZipPartitionsHelper.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: StateStoreAwareZipPartitionsHelper 3 | --- 4 | 5 | # StateStoreAwareZipPartitionsHelper Implicit Class 6 | 7 | `StateStoreAwareZipPartitionsHelper` is a Scala implicit class of a data RDD (of type `RDD[T]`) to [create a StateStoreAwareZipPartitionsRDD](#stateStoreAwareZipPartitions) to execute the following physical operators: 8 | 9 | * [FlatMapGroupsWithStateExec](../physical-operators/FlatMapGroupsWithStateExec.md) (with [hasInitialState](../physical-operators/FlatMapGroupsWithStateExec.md#hasInitialState) enabled) 10 | * [StreamingSymmetricHashJoinExec](../physical-operators/StreamingSymmetricHashJoinExec.md) 11 | 12 | ??? note "Implicit Class" 13 | [Implicit Classes](http://docs.scala-lang.org/overviews/core/implicit-classes.html) are a language feature in Scala for **implicit conversions** with **extension methods** for existing types. 14 | 15 | ## Creating StateStoreAwareZipPartitionsRDD 16 | 17 | ```scala 18 | stateStoreAwareZipPartitions[U: ClassTag, V: ClassTag]( 19 | dataRDD2: RDD[U], 20 | stateInfo: StatefulOperatorStateInfo, 21 | storeNames: Seq[String], 22 | storeCoordinator: StateStoreCoordinatorRef)( 23 | f: (Iterator[T], Iterator[U]) => Iterator[V]): RDD[V] 24 | ``` 25 | 26 | `stateStoreAwareZipPartitions` creates a new [StateStoreAwareZipPartitionsRDD](StateStoreAwareZipPartitionsRDD.md). 27 | 28 | --- 29 | 30 | `stateStoreAwareZipPartitions` is used when the following physical operators are executed: 31 | 32 | * [FlatMapGroupsWithStateExec](../physical-operators/FlatMapGroupsWithStateExec.md) (with [hasInitialState](../physical-operators/FlatMapGroupsWithStateExec.md#hasInitialState) enabled) 33 | * [StreamingSymmetricHashJoinExec](../physical-operators/StreamingSymmetricHashJoinExec.md) 34 | -------------------------------------------------------------------------------- /docs/kafka/CachedKafkaConsumer.md: -------------------------------------------------------------------------------- 1 | == [[CachedKafkaConsumer]] CachedKafkaConsumer 2 | 3 | CAUTION: FIXME 4 | 5 | === [[poll]] `poll` Internal Method 6 | 7 | CAUTION: FIXME 8 | 9 | === [[fetchData]] `fetchData` Internal Method 10 | 11 | CAUTION: FIXME 12 | -------------------------------------------------------------------------------- /docs/kafka/ConsumerStrategy.md: -------------------------------------------------------------------------------- 1 | # ConsumerStrategy 2 | 3 | `ConsumerStrategy` is an [abstraction](#contract) of [consumer strategies](#implementations) of which partitions to read records from. 4 | 5 | ## Contract 6 | 7 | ### Assigned TopicPartitions 8 | 9 | ```scala 10 | assignedTopicPartitions( 11 | admin: Admin): Set[TopicPartition] 12 | ``` 13 | 14 | Assigned `TopicPartition`s ([Apache Kafka]({{ kafka.api }}/org/apache/kafka/common/TopicPartition.html)) 15 | 16 | Used when: 17 | 18 | * `KafkaOffsetReaderAdmin` is requested to [fetchPartitionOffsets](KafkaOffsetReaderAdmin.md#fetchPartitionOffsets), [partitionsAssignedToAdmin](KafkaOffsetReaderAdmin.md#partitionsAssignedToAdmin) 19 | 20 | ### Creating Kafka Consumer 21 | 22 | ```scala 23 | createConsumer( 24 | kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] 25 | ``` 26 | 27 | Creates a `Consumer` ([Apache Kafka]({{ kafka.api }}/org/apache/kafka/clients/consumer/Consumer.html)) 28 | 29 | Used when: 30 | 31 | * `KafkaOffsetReaderConsumer` is requested for a [Kafka Consumer](KafkaOffsetReaderConsumer.md#consumer) 32 | 33 | ## Implementations 34 | 35 | ??? note "Sealed Trait" 36 | `ConsumerStrategy` is a Scala **sealed trait** which means that all of the implementations are in the same compilation unit (a single file). 37 | 38 | Learn more in the [Scala Language Specification]({{ scala.spec }}/05-classes-and-objects.html#sealed). 39 | 40 | * `AssignStrategy` 41 | * `SubscribePatternStrategy` 42 | * [SubscribeStrategy](SubscribeStrategy.md) 43 | -------------------------------------------------------------------------------- /docs/kafka/KafkaBatch.md: -------------------------------------------------------------------------------- 1 | # KafkaBatch 2 | 3 | `KafkaBatch` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaBatchInputPartition.md: -------------------------------------------------------------------------------- 1 | # KafkaBatchInputPartition 2 | 3 | `KafkaBatchInputPartition` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaContinuousInputPartition.md: -------------------------------------------------------------------------------- 1 | == [[KafkaContinuousInputPartition]] KafkaContinuousInputPartition 2 | 3 | `KafkaContinuousInputPartition` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaContinuousStream.md: -------------------------------------------------------------------------------- 1 | # KafkaContinuousStream 2 | 3 | `KafkaContinuousStream` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaDataConsumer.md: -------------------------------------------------------------------------------- 1 | == [[KafkaDataConsumer]] KafkaDataConsumer 2 | 3 | `KafkaDataConsumer` is the <> of <> that use <> that can be <>. 4 | 5 | [[contract]] 6 | .KafkaDataConsumer Contract (Abstract Methods Only) 7 | [cols="30m,70",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | internalConsumer 13 | a| [[internalConsumer]] 14 | 15 | [source, scala] 16 | ---- 17 | internalConsumer: InternalKafkaConsumer 18 | ---- 19 | 20 | Used when...FIXME 21 | 22 | | release 23 | a| [[release]] 24 | 25 | [source, scala] 26 | ---- 27 | release(): Unit 28 | ---- 29 | 30 | Used when...FIXME 31 | 32 | |=== 33 | 34 | [[implementations]] 35 | .KafkaDataConsumers 36 | [cols="30,70",options="header",width="100%"] 37 | |=== 38 | | KafkaDataConsumer 39 | | Description 40 | 41 | | CachedKafkaDataConsumer 42 | | [[CachedKafkaDataConsumer]] 43 | 44 | | NonCachedKafkaDataConsumer 45 | | [[NonCachedKafkaDataConsumer]] 46 | 47 | |=== 48 | 49 | === [[acquire]] Acquiring Cached KafkaDataConsumer for Partition -- `acquire` Object Method 50 | 51 | [source, scala] 52 | ---- 53 | acquire( 54 | topicPartition: TopicPartition, 55 | kafkaParams: ju.Map[String, Object], 56 | useCache: Boolean 57 | ): KafkaDataConsumer 58 | ---- 59 | 60 | `acquire`...FIXME 61 | 62 | NOTE: `acquire` is used when...FIXME 63 | 64 | === [[get]] Getting Kafka Record -- `get` Method 65 | 66 | [source, scala] 67 | ---- 68 | get( 69 | offset: Long, 70 | untilOffset: Long, 71 | pollTimeoutMs: Long, 72 | failOnDataLoss: Boolean 73 | ): ConsumerRecord[Array[Byte], Array[Byte]] 74 | ---- 75 | 76 | `get`...FIXME 77 | 78 | NOTE: `get` is used when...FIXME 79 | -------------------------------------------------------------------------------- /docs/kafka/KafkaMicroBatchInputPartition.md: -------------------------------------------------------------------------------- 1 | # KafkaMicroBatchInputPartition 2 | 3 | `KafkaMicroBatchInputPartition` is an `InputPartition` (of `InternalRows`) that is used (<>) exclusively when `KafkaMicroBatchReader` is requested for input partitions (when `DataSourceV2ScanExec` physical operator is requested for the partitions of the input RDD). 4 | 5 | [[creating-instance]] 6 | `KafkaMicroBatchInputPartition` takes the following to be created: 7 | 8 | * [[offsetRange]] [KafkaOffsetRange](KafkaOffsetRangeCalculator.md#KafkaOffsetRange) 9 | * [[executorKafkaParams]] Kafka parameters used for Kafka clients on executors (`Map[String, Object]`) 10 | * [[pollTimeoutMs]] Poll timeout (in ms) 11 | * [[failOnDataLoss]] `failOnDataLoss` flag 12 | * [[reuseKafkaConsumer]] `reuseKafkaConsumer` flag 13 | 14 | [[createPartitionReader]] 15 | `KafkaMicroBatchInputPartition` creates a [KafkaMicroBatchInputPartitionReader](KafkaMicroBatchInputPartitionReader.md) when requested for a `InputPartitionReader[InternalRow]` (as a part of the `InputPartition` contract). 16 | 17 | [[preferredLocations]] 18 | `KafkaMicroBatchInputPartition` simply requests the given <> for the optional `preferredLoc` when requested for `preferredLocations` (as a part of the `InputPartition` contract). 19 | -------------------------------------------------------------------------------- /docs/kafka/KafkaOffsetRangeLimit.md: -------------------------------------------------------------------------------- 1 | # KafkaOffsetRangeLimit — Desired Offset Range Limits 2 | 3 | `KafkaOffsetRangeLimit` represents the desired offset range limits for starting, ending, and specific offsets in [Kafka Data Source](index.md). 4 | 5 | [[implementations]] 6 | .KafkaOffsetRangeLimits 7 | [cols="1m,3",options="header",width="100%"] 8 | |=== 9 | | KafkaOffsetRangeLimit 10 | | Description 11 | 12 | | EarliestOffsetRangeLimit 13 | | [[EarliestOffsetRangeLimit]] Intent to bind to the *earliest* offset 14 | 15 | | LatestOffsetRangeLimit 16 | | [[LatestOffsetRangeLimit]] Intent to bind to the *latest* offset 17 | 18 | | SpecificOffsetRangeLimit 19 | a| [[SpecificOffsetRangeLimit]] Intent to bind to *specific offsets* with the following special offset "magic" numbers: 20 | 21 | * [[LATEST]] `-1` or `KafkaOffsetRangeLimit.LATEST` - the latest offset 22 | * [[EARLIEST]] `-2` or `KafkaOffsetRangeLimit.EARLIEST` - the earliest offset 23 | 24 | |=== 25 | 26 | NOTE: `KafkaOffsetRangeLimit` is a Scala *sealed trait* which means that all the <> are in the same compilation unit (a single file). 27 | 28 | `KafkaOffsetRangeLimit` is often used in a text-based representation and is converted to from *latest*, *earliest* or a *JSON-formatted text* using [KafkaSourceProvider.getKafkaOffsetRangeLimit](KafkaSourceProvider.md#getKafkaOffsetRangeLimit) utility. 29 | 30 | NOTE: A JSON-formatted text is of the following format `{"topicName":{"partition":offset},...}`, e.g. `{"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}`. 31 | 32 | `KafkaOffsetRangeLimit` is used when: 33 | 34 | * [KafkaRelation](KafkaRelation.md) is created (with the [starting](KafkaRelation.md#startingOffsets) and [ending](KafkaRelation.md#endingOffsets) offsets) 35 | 36 | * [KafkaSource](KafkaSource.md) is created (with the [starting offsets](KafkaRelation.md#startingOffsets)) 37 | 38 | * `KafkaSourceProvider` is requested to [convert configuration options to KafkaOffsetRangeLimits](KafkaSourceProvider.md#getKafkaOffsetRangeLimit) 39 | -------------------------------------------------------------------------------- /docs/kafka/KafkaOffsetReaderAdmin.md: -------------------------------------------------------------------------------- 1 | # KafkaOffsetReaderAdmin 2 | 3 | `KafkaOffsetReaderAdmin` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaOffsetReaderConsumer.md: -------------------------------------------------------------------------------- 1 | # KafkaOffsetReaderConsumer 2 | 3 | `KafkaOffsetReaderConsumer` is a [KafkaOffsetReader](KafkaOffsetReader.md). 4 | 5 | `KafkaOffsetReaderConsumer` is considered old and deprecated (in favour of the new `Admin`-based [KafkaOffsetReaderAdmin](KafkaOffsetReaderAdmin.md)). 6 | 7 | !!! note "spark.sql.streaming.kafka.useDeprecatedOffsetFetching" 8 | [spark.sql.streaming.kafka.useDeprecatedOffsetFetching](../configuration-properties.md#spark.sql.streaming.kafka.useDeprecatedOffsetFetching) configuration property controls what [KafkaOffsetReader](KafkaOffsetReader.md) is used. 9 | 10 | ## Creating Instance 11 | 12 | `KafkaOffsetReaderConsumer` takes the following to be created: 13 | 14 | * [ConsumerStrategy](ConsumerStrategy.md) 15 | * Driver Kafka Parameters 16 | * Reader Options 17 | * Driver GroupId Prefix 18 | 19 | `KafkaOffsetReaderConsumer` is created when: 20 | 21 | * `KafkaOffsetReader` is requested to [build a KafkaOffsetReader](KafkaOffsetReader.md#build) (with [spark.sql.streaming.kafka.useDeprecatedOffsetFetching](../configuration-properties.md#spark.sql.streaming.kafka.useDeprecatedOffsetFetching) property enabled) 22 | 23 | ## String Representation 24 | 25 | ```scala 26 | toString(): String 27 | ``` 28 | 29 | `toString` is part of the `Object` ([Java]({{ java.api }}/java/lang/Object.html#toString())) abstraction. 30 | 31 | --- 32 | 33 | `toString` requests the [ConsumerStrategy](#consumerStrategy) for the [string representation](ConsumerStrategy.md#toString). 34 | 35 | !!! note 36 | [ConsumerStrategy](ConsumerStrategy.md) does not overload `toString`, but the [implementations](ConsumerStrategy.md#implementations) do (e.g., [SubscribeStrategy](SubscribeStrategy.md#toString)). 37 | 38 | --- 39 | 40 | `toString` is used (among the other uses) when: 41 | 42 | * `KafkaMicroBatchStream` is requested for the [string representation](KafkaMicroBatchStream.md#toString) 43 | -------------------------------------------------------------------------------- /docs/kafka/KafkaSink.md: -------------------------------------------------------------------------------- 1 | # KafkaSink 2 | 3 | `KafkaSink` is a [streaming sink](../Sink.md) that [KafkaSourceProvider](KafkaSourceProvider.md) registers as the `kafka` format. 4 | 5 | ```text 6 | // start spark-shell or a Spark application with spark-sql-kafka-0-10 module 7 | // spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.3.0-SNAPSHOT 8 | import org.apache.spark.sql.SparkSession 9 | val spark: SparkSession = ... 10 | spark. 11 | readStream. 12 | format("text"). 13 | load("server-logs/*.out"). 14 | as[String]. 15 | writeStream. 16 | queryName("server-logs processor"). 17 | format("kafka"). // <-- uses KafkaSink 18 | option("topic", "topic1"). 19 | option("checkpointLocation", "/tmp/kafka-sink-checkpoint"). // <-- mandatory 20 | start 21 | 22 | // in another terminal 23 | $ echo hello > server-logs/hello.out 24 | 25 | // in the terminal with Spark 26 | FIXME 27 | ``` 28 | 29 | ## Creating Instance 30 | 31 | `KafkaSink` takes the following when created: 32 | 33 | * [[sqlContext]] `SQLContext` 34 | * [[executorKafkaParams]] Kafka parameters (used on executor) as a map of `(String, Object)` pairs 35 | * [[topic]] Optional topic name 36 | 37 | === [[addBatch]] `addBatch` Method 38 | 39 | [source, scala] 40 | ---- 41 | addBatch(batchId: Long, data: DataFrame): Unit 42 | ---- 43 | 44 | Internally, `addBatch` requests `KafkaWriter` to write the input `data` to the <> (if defined) or a topic in <>. 45 | 46 | `addBatch` is a part of [Sink](../Sink.md#addBatch) abstraction. 47 | -------------------------------------------------------------------------------- /docs/kafka/KafkaSourceInitialOffsetWriter.md: -------------------------------------------------------------------------------- 1 | # KafkaSourceInitialOffsetWriter 2 | 3 | `KafkaSourceInitialOffsetWriter` is a [HDFSMetadataLog](../HDFSMetadataLog.md) of [KafkaSourceOffset](KafkaSourceOffset.md)s. 4 | 5 | ## Creating Instance 6 | 7 | `KafkaSourceInitialOffsetWriter` takes the following to be created: 8 | 9 | * `SparkSession` ([Spark SQL]({{ book.spark_sql }}/SparkSession)) 10 | * Path of the metadata log directory 11 | 12 | `KafkaSourceInitialOffsetWriter` is created when: 13 | 14 | * `KafkaMicroBatchStream` is requested to [getOrCreateInitialPartitionOffsets](KafkaMicroBatchStream.md#getOrCreateInitialPartitionOffsets) 15 | * `KafkaSource` is requested to [getOrCreateInitialPartitionOffsets](KafkaSource.md#getOrCreateInitialPartitionOffsets) 16 | 17 | ## deserialize 18 | 19 | ```scala 20 | deserialize( 21 | in: InputStream): KafkaSourceOffset 22 | ``` 23 | 24 | `deserialize` is part of the [HDFSMetadataLog](../HDFSMetadataLog.md#deserialize) abstraction. 25 | 26 | --- 27 | 28 | `deserialize` creates a [KafkaSourceOffset](KafkaSourceOffset.md) (from a `SerializedOffset`). 29 | -------------------------------------------------------------------------------- /docs/kafka/KafkaStreamWriterFactory.md: -------------------------------------------------------------------------------- 1 | # KafkaStreamWriterFactory 2 | 3 | `KafkaStreamWriterFactory` is a [StreamingDataWriterFactory](../StreamingDataWriterFactory.md). 4 | 5 | ## Creating Instance 6 | 7 | `KafkaStreamWriterFactory` takes the following to be created: 8 | 9 | * Topic Name (optional) 10 | * `KafkaProducer` Parameters 11 | * Schema (`StructType`) 12 | 13 | `KafkaStreamWriterFactory` is created when: 14 | 15 | * `KafkaStreamingWrite` is requested for a [StreamingDataWriterFactory](KafkaStreamingWrite.md#createStreamingWriterFactory) 16 | 17 | ## createWriter 18 | 19 | ```scala 20 | createWriter( 21 | partitionId: Int, 22 | taskId: Long, 23 | epochId: Long): DataWriter[InternalRow] 24 | ``` 25 | 26 | `createWriter` is part of the [StreamingDataWriterFactory](../StreamingDataWriterFactory.md#createWriter) abstraction. 27 | 28 | --- 29 | 30 | `createWriter` creates a `KafkaDataWriter` ([Spark SQL]({{ book.spark_sql }}/kafka/KafkaDataWriter)) for the given [topic](#topic), [KafkaProducer parameters](#producerParams) and [schema](#schema). 31 | -------------------------------------------------------------------------------- /docs/kafka/KafkaStreamingWrite.md: -------------------------------------------------------------------------------- 1 | # KafkaStreamingWrite 2 | 3 | `KafkaStreamingWrite` is a [StreamingWrite](../StreamingWrite.md). 4 | 5 | ## Creating Instance 6 | 7 | `KafkaStreamingWrite` takes the following to be created: 8 | 9 | * Topic Name (Optional) 10 | * Kafka Producer Parameters 11 | * Schema 12 | 13 | `KafkaStreamingWrite` is created when: 14 | 15 | * `KafkaWrite` ([Spark SQL]({{ book.spark_sql }}/kafka/KafkaWrite)) is requested for a [StreamingWrite](../StreamingWrite.md) 16 | 17 | ## Creating StreamingDataWriterFactory 18 | 19 | ```scala 20 | createStreamingWriterFactory( 21 | info: PhysicalWriteInfo): KafkaStreamWriterFactory 22 | ``` 23 | 24 | `createStreamingWriterFactory` is part of the [StreamingWrite](../StreamingWrite.md#createStreamingWriterFactory) abstraction. 25 | 26 | --- 27 | 28 | `createStreamingWriterFactory` creates a [KafkaStreamWriterFactory](KafkaStreamWriterFactory.md) for the given [topic](#topic), [producerParams](#producerParams) and [schema](#schema). 29 | -------------------------------------------------------------------------------- /docs/kafka/KafkaTable.md: -------------------------------------------------------------------------------- 1 | # KafkaTable 2 | 3 | `KafkaTable` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/SubscribeStrategy.md: -------------------------------------------------------------------------------- 1 | # SubscribeStrategy 2 | 3 | `SubscribeStrategy` is a [ConsumerStrategy](ConsumerStrategy.md) that is used for [subscribe](options.md#subscribe) option. 4 | 5 | `SubscribeStrategy` uses [KafkaConsumer.subscribe]({{ kafka.api }}/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection)) to subscribe a [KafkaConsumer](#createConsumer) to the given [topics](#topics). 6 | 7 | ## Creating Instance 8 | 9 | `SubscribeStrategy` takes the following to be created: 10 | 11 | * Topic Names 12 | 13 | `SubscribeStrategy` is created when: 14 | 15 | * `KafkaSourceProvider` is requested for a [consumer strategy](KafkaSourceProvider.md#strategy) (for [subscribe](options.md#subscribe) option) 16 | 17 | ## Creating Kafka Consumer 18 | 19 | ```scala 20 | createConsumer( 21 | kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] 22 | ``` 23 | 24 | `createConsumer` is part of the [ConsumerStrategy](ConsumerStrategy.md#createConsumer) abstraction. 25 | 26 | --- 27 | 28 | `createConsumer` creates a `KafkaConsumer` ([Apache Kafka]({{ kafka.api }}/org/apache/kafka/clients/consumer/KafkaConsumer.html)) to subscribe (using [KafkaConsumer.subscribe]({{ kafka.api }}/org/apache/kafka/clients/consumer/KafkaConsumer.html#subscribe(java.util.Collection))) to the given [topics](#topics). 29 | 30 | ## Assigned TopicPartitions 31 | 32 | ```scala 33 | assignedTopicPartitions( 34 | admin: Admin): Set[TopicPartition] 35 | ``` 36 | 37 | `assignedTopicPartitions` is part of the [ConsumerStrategy](ConsumerStrategy.md#assignedTopicPartitions) abstraction. 38 | 39 | --- 40 | 41 | `assignedTopicPartitions` [retrieveAllPartitions](ConsumerStrategy.md#retrieveAllPartitions). 42 | 43 | ## String Representation 44 | 45 | ```scala 46 | toString(): String 47 | ``` 48 | 49 | `toString` is part of the `Object` ([Java]({{ java.api }}/java/lang/Object.html#toString())) abstraction. 50 | 51 | --- 52 | 53 | `toString` is the following (with the comma-separated [topic names](#topics)): 54 | 55 | ```text 56 | Subscribe[[topics]] 57 | ``` 58 | 59 | --- 60 | 61 | `toString` is used (among the other uses) when: 62 | 63 | * `KafkaOffsetReaderConsumer` is requested for the [string representation](KafkaOffsetReaderConsumer.md#toString) (for [KafkaMicroBatchStream](KafkaMicroBatchStream.md#toString)) 64 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/ResolveWriteToStream.md: -------------------------------------------------------------------------------- 1 | # ResolveWriteToStream Logical Resolution Rule 2 | 3 | `ResolveWriteToStream` is a logical resolution rule (`Rule[LogicalPlan]`) to resolve [WriteToStreamStatement](../logical-operators/WriteToStreamStatement.md) unary logical operators (into [WriteToStream](../logical-operators/WriteToStream.md)s). 4 | 5 | `ResolveWriteToStream` is part of `extendedResolutionRules` ([Spark SQL]({{ book.spark_sql }}/Analyzer#extendedResolutionRules)) of an `Analyzer` of the following: 6 | 7 | * `HiveSessionStateBuilder` ([Spark SQL]({{ book.spark_sql }}/hive/HiveSessionStateBuilder#analyzer)) 8 | * `BaseSessionStateBuilder` ([Spark SQL]({{ book.spark_sql }}/BaseSessionStateBuilder#analyzer)) 9 | 10 | ## Creating Instance 11 | 12 | `ResolveWriteToStream` takes no arguments to be created. 13 | 14 | ## Executing Rule 15 | 16 | ```scala 17 | apply( 18 | plan: LogicalPlan): LogicalPlan 19 | ``` 20 | 21 | `apply` is part of the `Rule` ([Spark SQL]({{ book.spark_sql }}/catalyst/Rule#apply)) abstraction. 22 | 23 | --- 24 | 25 | `apply` resolves [WriteToStreamStatement](../logical-operators/WriteToStreamStatement.md) unary logical operators into [WriteToStream](../logical-operators/WriteToStream.md)s. 26 | 27 | `apply` [resolveCheckpointLocation](#resolveCheckpointLocation). 28 | 29 | With [spark.sql.adaptive.enabled](../configuration-properties.md#spark.sql.adaptive.enabled) enabled, `apply`...FIXME 30 | 31 | With [spark.sql.streaming.unsupportedOperationCheck](../configuration-properties.md#spark.sql.streaming.unsupportedOperationCheck) enabled, `apply`...FIXME 32 | -------------------------------------------------------------------------------- /docs/logical-operators/ContinuousExecutionRelation.md: -------------------------------------------------------------------------------- 1 | # ContinuousExecutionRelation Leaf Logical Operator 2 | 3 | `ContinuousExecutionRelation` is a `MultiInstanceRelation` ([Spark SQL]({{ book.spark_sql }}/spark-sql-MultiInstanceRelation)) leaf logical operator. 4 | 5 | !!! tip 6 | Learn more about [Leaf Logical Operators]({{ book.spark_sql }}/logical-operators/LeafNode) in [The Internals of Spark SQL]({{ book.spark_sql }}) book. 7 | 8 | ## Creating Instance 9 | 10 | `ContinuousExecutionRelation` takes the following to be created: 11 | 12 | * [[source]] [ContinuousReadSupport](../continuous-execution/ContinuousReadSupport.md) source 13 | * [[extraOptions]] Options (`Map[String, String]`) 14 | * [[output]] Output attributes (`Seq[Attribute]`) 15 | * [[session]] `SparkSession` ([Spark SQL]({{ book.spark_sql }}/SparkSession)) 16 | 17 | `ContinuousExecutionRelation` is created (to represent [StreamingRelationV2](../logical-operators/StreamingRelationV2.md) with [ContinuousReadSupport](../continuous-execution/ContinuousReadSupport.md) data source) when `ContinuousExecution` is [created](../continuous-execution/ContinuousExecution.md) (and requested for the [logical plan](../continuous-execution/ContinuousExecution.md#logicalPlan)). 18 | -------------------------------------------------------------------------------- /docs/logical-operators/StreamingDataSourceV2Relation.md: -------------------------------------------------------------------------------- 1 | # StreamingDataSourceV2Relation Logical Operator 2 | 3 | `StreamingDataSourceV2Relation` is a leaf logical operator that represents [StreamingRelationV2](StreamingRelationV2.md) logical operator (with tables with a `SupportsRead` and `MICRO_BATCH_READ` or `CONTINUOUS_READ` capabilities) at execution time. 4 | 5 | !!! tip 6 | Learn more about [Leaf Logical Operators]({{ book.spark_sql }}/logical-operators/LeafNode), [SupportsRead]({{ book.spark_sql }}/connector/SupportsRead) and [Table Capabilities]({{ book.spark_sql }}/connector/TableCapability) in [The Internals of Spark SQL]({{ book.spark_sql }}) online book. 7 | 8 | ## Creating Instance 9 | 10 | `StreamingDataSourceV2Relation` takes the following to be created: 11 | 12 | * Output Attributes ([Spark SQL]({{ book.spark_sql }}/expressions/Attribute)) 13 | * `Scan` ([Spark SQL]({{ book.spark_sql }}/connector/Scan)) 14 | * [SparkDataStream](../SparkDataStream.md) 15 | * Start [Offset](../Offset.md) (default: undefined) 16 | * End [Offset](../Offset.md) (default: undefined) 17 | 18 | `StreamingDataSourceV2Relation` is created when: 19 | 20 | * `MicroBatchExecution` stream execution engine is requested for an [analyzed logical query plan](../micro-batch-execution/MicroBatchExecution.md#logicalPlan) (for [StreamingRelationV2](StreamingRelationV2.md) with a `SupportsRead` table with `MICRO_BATCH_READ` capability) 21 | 22 | * `ContinuousExecution` stream execution engine is requested for an [analyzed logical query plan](../continuous-execution/ContinuousExecution.md#logicalPlan) (for [StreamingRelationV2](StreamingRelationV2.md) with a `SupportsRead` table with `CONTINUOUS_READ` capability) 23 | 24 | ## Computing Stats 25 | 26 | ```scala 27 | computeStats(): Statistics 28 | ``` 29 | 30 | For [Scans](#scan) with `SupportsReportStatistics`, `computeStats` requests the scan to `estimateStatistics`. 31 | 32 | !!! tip 33 | Learn more about [Scan]({{ book.spark_sql }}/connector/Scan) and [SupportsReportStatistics]({{ book.spark_sql }}/connector/SupportsReportStatistics) in [The Internals of Spark SQL]({{ book.spark_sql }}) online book. 34 | 35 | For other types of scans, `computeStats` simply assumes the default size and no row count. 36 | 37 | `computeStats` is part of the `LeafNode` abstraction. 38 | -------------------------------------------------------------------------------- /docs/logical-operators/StreamingExecutionRelation.md: -------------------------------------------------------------------------------- 1 | # StreamingExecutionRelation Leaf Logical Operator 2 | 3 | `StreamingExecutionRelation` is a leaf logical operator ([Spark SQL]({{ book.spark_sql }}/logical-operators/LeafNode)) that represents a [streaming source](../Source.md) in the logical query plan of a streaming query. 4 | 5 | The main use of `StreamingExecutionRelation` logical operator is to be a "placeholder" in a logical query plan that will be replaced with the real relation (with new data that has arrived since the last batch) or an empty `LocalRelation` when `StreamExecution` is requested to [transforming logical plan to include the Sources and MicroBatchReaders with new data](../micro-batch-execution/MicroBatchExecution.md#runBatch-newBatchesPlan). 6 | 7 | !!! note 8 | Right after `StreamExecution` [has started running streaming batches](../micro-batch-execution/MicroBatchExecution.md#runStream-initializing-sources) it initializes the streaming sources by transforming the analyzed logical plan of the streaming query so that every [StreamingRelation](StreamingRelation.md) logical operator is replaced by the corresponding `StreamingExecutionRelation`. 9 | 10 | ![StreamingExecutionRelation Represents Streaming Source At Execution](../images/StreamingExecutionRelation.png) 11 | 12 | !!! note 13 | `StreamingExecutionRelation` is also resolved (_planned_) to a [StreamingRelationExec](../physical-operators/StreamingRelationExec.md) physical operator in [StreamingRelationStrategy](../execution-planning-strategies/StreamingRelationStrategy.md) execution planning strategy only when [explaining](../operators/explain.md) a streaming `Dataset`. 14 | 15 | ## Creating Instance 16 | 17 | `StreamingExecutionRelation` takes the following to be created: 18 | 19 | * `BaseStreamingSource` 20 | * Output Attributes (`Seq[Attribute]`) 21 | * `SparkSession` 22 | 23 | `StreamingExecutionRelation` is created when: 24 | 25 | * `MicroBatchExecution` stream execution engine is requested for the [analyzed logical query plan](../micro-batch-execution/MicroBatchExecution.md#logicalPlan) (for every [StreamingRelation](StreamingRelation.md)) 26 | -------------------------------------------------------------------------------- /docs/logical-operators/StreamingRelationV2.md: -------------------------------------------------------------------------------- 1 | # StreamingRelationV2 Leaf Logical Operator 2 | 3 | `StreamingRelationV2` is a leaf logical operator that represents `SupportsRead` streaming tables (with `MICRO_BATCH_READ` or `CONTINUOUS_READ` capabilities) in a logical plan of a streaming query. 4 | 5 | !!! tip 6 | Learn more about [Leaf Logical Operators]({{ book.spark_sql }}/logical-operators/LeafNode), [SupportsRead]({{ book.spark_sql }}/connector/SupportsRead) and [Table Capabilities]({{ book.spark_sql }}/connector/TableCapability) in [The Internals of Spark SQL]({{ book.spark_sql }}) online book. 7 | 8 | ## Creating Instance 9 | 10 | `StreamingRelationV2` takes the following to be created: 11 | 12 | * `TableProvider` ([Spark SQL]({{ book.spark_sql }}/connector/TableProvider)) 13 | * Source Name 14 | * `Table` ([Spark SQL]({{ book.spark_sql }}/connector/Table)) 15 | * Extra Options 16 | * Output Attributes ([Spark SQL]({{ book.spark_sql }}/expressions/Attribute)) 17 | * [StreamingRelation](StreamingRelation.md) 18 | * `SparkSession` ([Spark SQL]({{ book.spark_sql }}/SparkSession)) 19 | 20 | `StreamingRelationV2` is created when: 21 | 22 | * `DataStreamReader` is reqested to [load data](../DataStreamReader.md#load) (for a `SupportsRead` table with `MICRO_BATCH_READ` or `CONTINUOUS_READ` capabilities) 23 | * `MemoryStreamBase` is requested for a [logical query plan](../datasources/memory/MemoryStreamBase.md#logicalPlan) 24 | 25 | ## Logical Resolution 26 | 27 | `StreamingRelationV2` is resolved to the following leaf logical operators: 28 | 29 | * [StreamingDataSourceV2Relation](StreamingDataSourceV2Relation.md) or [StreamingExecutionRelation](StreamingExecutionRelation.md) when `MicroBatchExecution` stream execution engine is requested for an [analyzed logical plan](../micro-batch-execution/MicroBatchExecution.md#logicalPlan) 30 | * [StreamingDataSourceV2Relation](StreamingDataSourceV2Relation.md) when `ContinuousExecution` stream execution engine is created (and initializes an [analyzed logical plan](../continuous-execution/ContinuousExecution.md#logicalPlan)) 31 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteToContinuousDataSource.md: -------------------------------------------------------------------------------- 1 | # WriteToContinuousDataSource Unary Logical Operator 2 | 3 | `WriteToContinuousDataSource` is a unary logical operator (`LogicalPlan`) that is created when `ContinuousExecution` is requested to [run a streaming query in continuous mode](../continuous-execution/ContinuousExecution.md#runContinuous) (to create an [IncrementalExecution](../IncrementalExecution.md)). 4 | 5 | `WriteToContinuousDataSource` is planned (_translated_) to a [WriteToContinuousDataSourceExec](../physical-operators/WriteToContinuousDataSourceExec.md) unary physical operator (when `DataSourceV2Strategy` execution planning strategy is requested to plan a logical query). 6 | 7 | [[output]] 8 | `WriteToContinuousDataSource` uses empty output schema (which is exactly to say that no output is expected whatsoever). 9 | 10 | ## Creating Instance 11 | 12 | `WriteToContinuousDataSource` takes the following to be created: 13 | 14 | * [[query]] Child logical operator (`LogicalPlan`) 15 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteToDataSourceV2.md: -------------------------------------------------------------------------------- 1 | # WriteToDataSourceV2 Logical Operator 2 | 3 | `WriteToDataSourceV2` is a unary logical operator ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan#UnaryNode)) that represents [WriteToMicroBatchDataSource](WriteToMicroBatchDataSource.md) unary logical operator at logical optimization time for writing data in Spark Structured Streaming. 4 | 5 | !!! note "Deprecated" 6 | `WriteToDataSourceV2` is deprecated since Spark SQL 2.4.0 (in favour of `AppendData` logical operator and alike). 7 | 8 | ## Creating Instance 9 | 10 | `WriteToDataSourceV2` takes the following to be created: 11 | 12 | * `DataSourceV2Relation` ([Spark SQL]({{ book.spark_sql }}/logical-operators/DataSourceV2Relation)) 13 | * `BatchWrite` ([Spark SQL]({{ book.spark_sql }}/connector/BatchWrite)) 14 | * Logical Query Plan ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan)) 15 | * Write `CustomMetric`s ([Spark SQL]({{ book.spark_sql }}/connector/CustomMetric)) 16 | 17 | `WriteToDataSourceV2` is created when: 18 | 19 | * `V2Writes` ([Spark SQL]({{ book.spark_sql }}/logical-optimizations/V2Writes)) logical optimization is requested to optimize a logical query (with a [WriteToMicroBatchDataSource](WriteToMicroBatchDataSource.md) unary logical operator) 20 | 21 | ## Query Planning 22 | 23 | `WriteToDataSourceV2` is planned as [WriteToDataSourceV2Exec](../physical-operators/WriteToDataSourceV2Exec.md) physical operator by `DataSourceV2Strategy` ([Spark SQL]({{ book.spark_sql }}/execution-planning-strategies/DataSourceV2Strategy)) execution planning strategy. 24 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteToMicroBatchDataSource.md: -------------------------------------------------------------------------------- 1 | # WriteToMicroBatchDataSource Logical Operator 2 | 3 | `WriteToMicroBatchDataSource` is a unary logical operator ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan#UnaryNode)) that is a top-level operator in the [analyzed logical query plan](../micro-batch-execution/MicroBatchExecution.md#logicalPlan) of a streaming query with a [sink](../StreamExecution.md#sink) that `SupportsWrite` ([Spark SQL]({{ book.spark_sql }}/connector/SupportsWrite)). 4 | 5 | ## Creating Instance 6 | 7 | `WriteToMicroBatchDataSource` takes the following to be created: 8 | 9 | * `DataSourceV2Relation` ([Spark SQL]({{ book.spark_sql }}/logical-operators/DataSourceV2Relation)) leaf logical operator 10 | * Table with `SupportsWrite` ([Spark SQL]({{ book.spark_sql }}/connector/SupportsWrite)) 11 | * Query `LogicalPlan` ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan)) 12 | * Query ID 13 | * Write Options (`Map[String, String]`) 14 | * [OutputMode](../OutputMode.md) 15 | * Batch ID 16 | 17 | `WriteToMicroBatchDataSource` is created when: 18 | 19 | * `MicroBatchExecution` is requested for the [analyzed logical query plan](../micro-batch-execution/MicroBatchExecution.md#logicalPlan) (with a [sink](../StreamExecution.md#sink) that `SupportsWrite` ([Spark SQL]({{ book.spark_sql }}/connector/SupportsWrite))) 20 | 21 | ## Query Planning 22 | 23 | `WriteToMicroBatchDataSource` is optimized to [WriteToDataSourceV2](WriteToDataSourceV2.md) logical operator by `V2Writes` ([Spark SQL]({{ book.spark_sql }}/logical-optimizations/V2Writes)) logical optimization. 24 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteToStream.md: -------------------------------------------------------------------------------- 1 | # WriteToStream Logical Operator 2 | 3 | `WriteToStream` is a unary logical operator ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan#UnaryNode)) that represents [WriteToStreamStatement](WriteToStreamStatement.md) operator at execution time (of a streaming query). 4 | 5 | `WriteToStream` is used to create the stream execution engines: 6 | 7 | * [MicroBatchExecution](../micro-batch-execution/MicroBatchExecution.md#plan) 8 | * [ContinuousExecution](../continuous-execution/ContinuousExecution.md#plan) 9 | 10 | ## Creating Instance 11 | 12 | `WriteToStream` takes the following to be created: 13 | 14 | * Name 15 | * Checkpoint Location 16 | * Sink `Table` ([Spark SQL]({{ book.spark_sql }}/connector/Table/)) 17 | * [OutputMode](../OutputMode.md) 18 | * `deleteCheckpointOnStop` flag 19 | * `LogicalPlan` ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan)) of the Input Query 20 | * Optional `TableCatalog` and `Identifier` (default: undefined) 21 | 22 | `WriteToStream` is created when: 23 | 24 | * [ResolveWriteToStream](../logical-analysis-rules/ResolveWriteToStream.md) logical resolution rule is executed (to resolve [WriteToStreamStatement](WriteToStreamStatement.md) operators) 25 | 26 | ## isStreaming 27 | 28 | ```scala 29 | isStreaming: Boolean 30 | ``` 31 | 32 | `isStreaming` is part of the `LogicalPlan` ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan#isStreaming)) abstraction. 33 | 34 | --- 35 | 36 | `isStreaming` is `true`. 37 | 38 | ## Output Schema 39 | 40 | ```scala 41 | output: Seq[Attribute] 42 | ``` 43 | 44 | `output` is part of the `QueryPlan` ([Spark SQL]({{ book.spark_sql }}/catalyst/QueryPlan#output)) abstraction. 45 | 46 | --- 47 | 48 | `output` is empty (`Nil`) 49 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteToStreamStatement.md: -------------------------------------------------------------------------------- 1 | # WriteToStreamStatement Logical Operator 2 | 3 | `WriteToStreamStatement` is a unary logical operator ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan#UnaryNode)). 4 | 5 | ## Creating Instance 6 | 7 | `WriteToStreamStatement` takes the following to be created: 8 | 9 | * User-specified Name 10 | * Checkpoint Location 11 | * `useTempCheckpointLocation` flag 12 | * `recoverFromCheckpointLocation` flag 13 | * Sink `Table` ([Spark SQL]({{ book.spark_sql }}/connector/Table/)) 14 | * [OutputMode](../OutputMode.md) 15 | * Hadoop `Configuration` 16 | * `isContinuousTrigger` flag 17 | * `LogicalPlan` ([Spark SQL]({{ book.spark_sql }}/logical-operators/LogicalPlan)) of the Input Query 18 | * Optional `TableCatalog` and `Identifier` (default: undefined) 19 | 20 | `WriteToStreamStatement` is created when: 21 | 22 | * `StreamingQueryManager` is requested to [create a streaming query](../StreamingQueryManager.md#createQuery) 23 | 24 | ## Logical Resolution 25 | 26 | `WriteToStreamStatement` is resolved to [WriteToStream](WriteToStream.md) operator using [ResolveWriteToStream](../logical-analysis-rules/ResolveWriteToStream.md) logical resolution rule. 27 | -------------------------------------------------------------------------------- /docs/micro-batch-execution/MicroBatchWrite.md: -------------------------------------------------------------------------------- 1 | # MicroBatchWrite 2 | 3 | `MicroBatchWrite` is a `BatchWrite` ([Spark SQL]({{ book.spark_sql }}/connector/BatchWrite)) for [WriteToDataSourceV2](../logical-operators/WriteToDataSourceV2.md) logical operator in [Micro-Batch Stream Processing](index.md). 4 | 5 | !!! note "WriteToMicroBatchDataSource" 6 | `WriteToDataSourceV2` logical operator replaces [WriteToMicroBatchDataSource](../logical-operators/WriteToMicroBatchDataSource.md) logical operator at logical optimization (using `V2Writes` logical optimization). 7 | 8 | `MicroBatchWrite` is just a very thin wrapper over [StreamingWrite](#writeSupport) and does nothing but delegates (_relays_) all the important execution-specific calls to it. 9 | 10 | ## Creating Instance 11 | 12 | `MicroBatchWrite` takes the following to be created: 13 | 14 | * Epoch ID 15 | * [StreamingWrite](../StreamingWrite.md) 16 | 17 | `MicroBatchWrite` is created when: 18 | 19 | * `V2Writes` ([Spark SQL]({{ book.spark_sql }}/logical-optimizations/V2Writes)) logical optimization is requested to optimize a logical plan (with a [WriteToMicroBatchDataSource](../logical-operators/WriteToMicroBatchDataSource.md)) 20 | 21 | ## Committing Writing Job 22 | 23 | ```scala 24 | commit( 25 | messages: Array[WriterCommitMessage]): Unit 26 | ``` 27 | 28 | `commit` is part of the `BatchWrite` ([Spark SQL]({{ book.spark_sql }}/connector/BatchWrite#commit)) abstraction. 29 | 30 | --- 31 | 32 | `commit` requests the [StreamingWrite](#writeSupport) to [commit](../StreamingWrite.md#commit). 33 | 34 | ## Creating DataWriterFactory for Batch Write 35 | 36 | ```scala 37 | createBatchWriterFactory( 38 | info: PhysicalWriteInfo): DataWriterFactory 39 | ``` 40 | 41 | `createBatchWriterFactory` is part of the `BatchWrite` ([Spark SQL]({{ book.spark_sql }}/connector/BatchWrite#createBatchWriterFactory)) abstraction. 42 | 43 | --- 44 | 45 | `createBatchWriterFactory` requests the [StreamingWrite](#writeSupport) to [create a StreamingDataWriterFactory](../StreamingWrite.md#createStreamingWriterFactory). 46 | 47 | In the end, `createBatchWriterFactory` creates a [MicroBatchWriterFactory](MicroBatchWriterFactory.md) (with the given [epochId](#epochId) and the `StreamingDataWriterFactory`). 48 | -------------------------------------------------------------------------------- /docs/micro-batch-execution/MicroBatchWriterFactory.md: -------------------------------------------------------------------------------- 1 | # MicroBatchWriterFactory 2 | 3 | `MicroBatchWriterFactory` is a `DataWriterFactory` ([Spark SQL]({{ book.spark_sql }}/connector/DataWriterFactory)). 4 | 5 | ## Creating Instance 6 | 7 | `MicroBatchWriterFactory` takes the following to be created: 8 | 9 | * Epoch ID 10 | * [StreamingDataWriterFactory](../StreamingDataWriterFactory.md) 11 | 12 | `MicroBatchWriterFactory` is created when: 13 | 14 | * `MicroBatchWrite` is requested to [create a DataWriterFactory for batch write](MicroBatchWrite.md#createBatchWriterFactory) 15 | 16 | ## Creating DataWriter 17 | 18 | ```scala 19 | createWriter( 20 | partitionId: Int, 21 | taskId: Long): DataWriter[InternalRow] 22 | ``` 23 | 24 | `createWriter` is part of the `DataWriterFactory` ([Spark SQL]({{ book.spark_sql }}/connector/DataWriterFactory#createWriter)) abstraction. 25 | 26 | --- 27 | 28 | `createWriter` requests the [StreamingDataWriterFactory](#streamingWriterFactory) for a [DataWriter](../StreamingDataWriterFactory.md#createWriter). 29 | -------------------------------------------------------------------------------- /docs/monitoring/ExecutionStats.md: -------------------------------------------------------------------------------- 1 | == [[ExecutionStats]] ExecutionStats 2 | 3 | `ExecutionStats` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/monitoring/SinkProgress.md: -------------------------------------------------------------------------------- 1 | # SinkProgress 2 | 3 | `SinkProgress` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/monitoring/SourceProgress.md: -------------------------------------------------------------------------------- 1 | # SourceProgress 2 | 3 | `SourceProgress` is a `Serializable` representation of progress of a [SparkDataStream](../SparkDataStream.md) (in the execution of a [StreamingQuery](../StreamingQuery.md) during a trigger). 4 | 5 | ## Creating Instance 6 | 7 | `SourceProgress` takes the following to be created: 8 | 9 | * `description` 10 | * `startOffset` 11 | * `endOffset` 12 | * `latestOffset` 13 | * `numInputRows` 14 | * `inputRowsPerSecond` 15 | * `processedRowsPerSecond` 16 | * [Metrics](#metrics) 17 | 18 | `SourceProgress` is created when: 19 | 20 | * `ProgressReporter` is requested to [finishTrigger](../ProgressReporter.md#finishTrigger) (and [update stream progress](../ProgressReporter.md#updateProgress) with a new [StreamingQueryProgress](StreamingQueryProgress.md#sources)) 21 | 22 | ### Metrics 23 | 24 | ```java 25 | metrics: Map[String, String] 26 | ``` 27 | 28 | `SourceProgress` is given a `metrics` that is the [metrics](../ReportsSourceMetrics.md#metrics) for this progress of [ReportsSourceMetrics](../ReportsSourceMetrics.md) data stream. 29 | 30 | `metrics` is used in [jsonValue](#jsonValue). 31 | 32 | ## Converting to JSON Representation 33 | 34 | ```scala 35 | jsonValue: JValue 36 | ``` 37 | 38 | `jsonValue` converts this `SourceProgress` to JSON representation. 39 | 40 | --- 41 | 42 | `jsonValue` is used when: 43 | 44 | * `StreamingQueryProgress` is requested for [jsonValue](StreamingQueryProgress.md#jsonValue) 45 | * `SourceProgress` is requested to [json](#json), [prettyJson](#prettyJson) 46 | -------------------------------------------------------------------------------- /docs/monitoring/StreamingQueryStatus.md: -------------------------------------------------------------------------------- 1 | # StreamingQueryStatus 2 | 3 | `StreamingQueryStatus` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/operators/crossJoin.md: -------------------------------------------------------------------------------- 1 | # crossJoin Operator — Streaming Join 2 | 3 | ```scala 4 | crossJoin( 5 | right: Dataset[_]): DataFrame 6 | ``` 7 | 8 | `crossJoin` operator...FIXME 9 | -------------------------------------------------------------------------------- /docs/operators/explain.md: -------------------------------------------------------------------------------- 1 | # Dataset.explain Operator — Explaining Streaming Queries 2 | 3 | ```scala 4 | explain(): Unit // <1> 5 | explain( 6 | extended: Boolean): Unit 7 | ``` 8 | <1> Calls `explain` with `extended` flag disabled 9 | 10 | `Dataset.explain` operator explains query plans, i.e. prints the logical and (with `extended` flag enabled) physical query plans to the console. 11 | 12 | Internally, `explain` creates a `ExplainCommand` runnable command with the logical plan and `extended` flag. 13 | 14 | `explain` then executes the plan with `ExplainCommand` runnable command and collects the results that are printed out to the standard output. 15 | 16 | [NOTE] 17 | ==== 18 | `explain` uses `SparkSession` to access the current `SessionState` to execute the plan. 19 | 20 | [source, scala] 21 | ---- 22 | import org.apache.spark.sql.execution.command.ExplainCommand 23 | val explain = ExplainCommand(...) 24 | spark.sessionState.executePlan(explain) 25 | ---- 26 | ==== 27 | 28 | For streaming Datasets, `ExplainCommand` command simply creates a [IncrementalExecution](../IncrementalExecution.md) for the `SparkSession` and the logical plan. 29 | 30 | NOTE: For the purpose of `explain`, `IncrementalExecution` is created with the output mode `Append`, checkpoint location ``, run id a random number, current batch id `0` and offset metadata empty. They do not really matter when explaining the load-part of a streaming query. 31 | 32 | ## Demo 33 | 34 | ```text 35 | val records = spark. 36 | readStream. 37 | format("rate"). 38 | load 39 | scala> records.explain 40 | == Physical Plan == 41 | StreamingRelation rate, [timestamp#0, value#1L] 42 | 43 | scala> records.explain(extended = true) 44 | == Parsed Logical Plan == 45 | StreamingRelation DataSource(org.apache.spark.sql.SparkSession@4071aa13,rate,List(),None,List(),None,Map(),None), rate, [timestamp#0, value#1L] 46 | 47 | == Analyzed Logical Plan == 48 | timestamp: timestamp, value: bigint 49 | StreamingRelation DataSource(org.apache.spark.sql.SparkSession@4071aa13,rate,List(),None,List(),None,Map(),None), rate, [timestamp#0, value#1L] 50 | 51 | == Optimized Logical Plan == 52 | StreamingRelation DataSource(org.apache.spark.sql.SparkSession@4071aa13,rate,List(),None,List(),None,Map(),None), rate, [timestamp#0, value#1L] 53 | 54 | == Physical Plan == 55 | StreamingRelation rate, [timestamp#0, value#1L] 56 | ``` 57 | -------------------------------------------------------------------------------- /docs/operators/flatMapGroupsWithState.md: -------------------------------------------------------------------------------- 1 | # flatMapGroupsWithState Operator 2 | 3 | `flatMapGroupsWithState` is part of `KeyValueGroupedDataset` ([Spark SQL]({{ book.spark_sql }}/basic-aggregation/KeyValueGroupedDataset)) API for [Arbitrary Stateful Streaming Aggregation](../arbitrary-stateful-streaming-aggregation/index.md) with an explicit state logic. 4 | 5 | ```scala 6 | flatMapGroupsWithState[S: Encoder, U: Encoder]( 7 | outputMode: OutputMode, 8 | timeoutConf: GroupStateTimeout)( 9 | func: (K, Iterator[V], GroupState[S]) => Iterator[U]): Dataset[U] 10 | flatMapGroupsWithState[S: Encoder, U: Encoder]( 11 | outputMode: OutputMode, 12 | timeoutConf: GroupStateTimeout, 13 | initialState: KeyValueGroupedDataset[K, S])( 14 | func: (K, Iterator[V], GroupState[S]) => Iterator[U]): Dataset[U] // (1)! 15 | ``` 16 | 17 | 1. Since 3.2.0 18 | 19 | ## Input Arguments 20 | 21 | `flatMapGroupsWithState` accepts the following: 22 | 23 | * [OutputMode](../OutputMode.md) 24 | * [GroupStateTimeout](../GroupStateTimeout.md) 25 | * A function with the `K` key and the `V` values and the current [GroupState](../GroupState.md) for the given `K` key 26 | * (optionally) `KeyValueGroupedDataset` for an user-defined initial state 27 | 28 | ## FlatMapGroupsWithState Logical Operator 29 | 30 | `flatMapGroupsWithState` creates a `Dataset` with [FlatMapGroupsWithState](../logical-operators/FlatMapGroupsWithState.md) logical operator with the following: 31 | 32 | * `LogicalGroupState` 33 | * `groupingAttributes` of the `KeyValueGroupedDataset` 34 | * `dataAttributes` of the `KeyValueGroupedDataset` 35 | * `isMapGroupsWithState` flag disabled (`false`) 36 | * `logicalPlan` of the `KeyValueGroupedDataset` as the [child](../logical-operators/FlatMapGroupsWithState.md#child) operator 37 | 38 | ## Output Modes 39 | 40 | `flatMapGroupsWithState` supports [Append](../OutputMode.md#Append) and [Update](../OutputMode.md#Update) output modes only and throws an `IllegalArgumentException` otherwise: 41 | 42 | ```text 43 | The output mode of function should be append or update 44 | ``` 45 | 46 | !!! note 47 | An `OutputMode` is a required argument, but does not seem to be used at all. Check out the question [What's the purpose of OutputMode in flatMapGroupsWithState? How/where is it used?](https://stackoverflow.com/q/56921772/1305344) on StackOverflow. 48 | -------------------------------------------------------------------------------- /docs/operators/index.md: -------------------------------------------------------------------------------- 1 | # Streaming Operators — High-Level Declarative Streaming Dataset API 2 | 3 | Dataset API defines a set of operators that are used in Spark Structured Streaming and together constitute the **High-Level Declarative Streaming Dataset API**. 4 | -------------------------------------------------------------------------------- /docs/operators/join.md: -------------------------------------------------------------------------------- 1 | # join Operator 2 | 3 | `join` is part of `Dataset` API for [Streaming Join](../join/index.md). 4 | 5 | ```scala 6 | join( 7 | right: Dataset[_]): DataFrame 8 | join( 9 | right: Dataset[_], 10 | joinExprs: Column): DataFrame 11 | join( 12 | right: Dataset[_], 13 | joinExprs: Column, 14 | joinType: String): DataFrame 15 | join( 16 | right: Dataset[_], 17 | usingColumns: Seq[String]): DataFrame 18 | join( 19 | right: Dataset[_], 20 | usingColumns: Seq[String], 21 | joinType: String): DataFrame 22 | join( 23 | right: Dataset[_], 24 | usingColumn: String): DataFrame 25 | ``` 26 | 27 | `join` creates a `Dataset` with a `Join` ([Spark SQL]({{ book.spark_sql }}/logical-operators/Join)) logical operator. 28 | -------------------------------------------------------------------------------- /docs/operators/joinWith.md: -------------------------------------------------------------------------------- 1 | # joinWith Operator — Streaming Join 2 | 3 | ```scala 4 | joinWith[U]( 5 | other: Dataset[U], 6 | condition: Column): Dataset[(T, U)] 7 | joinWith[U]( 8 | other: Dataset[U], 9 | condition: Column, 10 | joinType: String): Dataset[(T, U)] 11 | ``` 12 | 13 | [Streaming Join](../join/index.md) 14 | -------------------------------------------------------------------------------- /docs/operators/withWatermark.md: -------------------------------------------------------------------------------- 1 | # withWatermark Operator — Event-Time Watermark 2 | 3 | ```scala 4 | withWatermark( 5 | eventTime: String, 6 | delayThreshold: String): Dataset[T] 7 | ``` 8 | 9 | `withWatermark` specifies a [streaming watermark](../watermark/index.md) (on the given `eventTime` column with a delay threshold). 10 | 11 | `withWatermark` specifies the `eventTime` column for **event time watermark** and `delayThreshold` for **event lateness**. 12 | 13 | `eventTime` specifies the column to use for watermark and can be either part of `Dataset` from the source or custom-generated using `current_time` or `current_timestamp` functions. 14 | 15 | !!! note 16 | **Watermark** tracks a point in time before which it is assumed no more late events are supposed to arrive (and if they have, the late events are considered really late and simply dropped). 17 | 18 | !!! note 19 | Spark Structured Streaming uses watermark for the following: 20 | 21 | * To know when a given time window aggregation (using [groupBy](groupBy.md) operator with [window](window.md) standard function) can be finalized and thus emitted when using output modes that do not allow updates, like [Append](../OutputMode.md#Append) output mode. 22 | 23 | * To minimize the amount of state that we need to keep for ongoing aggregations, e.g. [mapGroupsWithState](../operators/mapGroupsWithState.md) (for implicit state management), [flatMapGroupsWithState](flatMapGroupsWithState.md) (for user-defined state management) and [dropDuplicates](dropDuplicates.md) operators. 24 | 25 | The **current watermark** is computed by looking at the maximum `eventTime` seen across all of the partitions in a query minus a user-specified `delayThreshold`. Due to the cost of coordinating this value across partitions, the actual watermark used is only guaranteed to be at least `delayThreshold` behind the actual event time. 26 | 27 | !!! note 28 | In some cases Spark may still process records that arrive more than `delayThreshold` late. 29 | -------------------------------------------------------------------------------- /docs/operators/writeStream.md: -------------------------------------------------------------------------------- 1 | # writeStream Operator 2 | 3 | ```scala 4 | writeStream: DataStreamWriter[T] 5 | ``` 6 | 7 | `writeStream` creates a [DataStreamWriter](../DataStreamWriter.md) for persisting the result of a streaming query to an external data system 8 | -------------------------------------------------------------------------------- /docs/physical-operators/SessionWindowStateStoreRestoreExec.md: -------------------------------------------------------------------------------- 1 | # SessionWindowStateStoreRestoreExec Physical Operator 2 | 3 | ## Required Child Output Distribution 4 | 5 | ```scala 6 | requiredChildDistribution: Seq[Distribution] 7 | ``` 8 | 9 | `requiredChildDistribution` is part of the `SparkPlan` ([Spark SQL]({{ book.spark_sql }}/physical-operators/SparkPlan/#requiredChildDistribution)) abstraction. 10 | 11 | --- 12 | 13 | `requiredChildDistribution`...FIXME 14 | -------------------------------------------------------------------------------- /docs/physical-operators/SessionWindowStateStoreSaveExec.md: -------------------------------------------------------------------------------- 1 | # SessionWindowStateStoreSaveExec Physical Operator 2 | 3 | `SessionWindowStateStoreSaveExec` is a unary physical operator ([Spark SQL]({{ book.spark_sql }}/physical-operators/UnaryExecNode/)). 4 | 5 | ## Short Name 6 | 7 | ```scala 8 | shortName: String 9 | ``` 10 | 11 | `shortName` is part of the [StateStoreWriter](StateStoreWriter.md#shortName) abstraction. 12 | 13 | --- 14 | 15 | `shortName` is the following text: 16 | 17 | ```text 18 | sessionWindowStateStoreSave 19 | ``` 20 | 21 | ## Required Child Output Distribution 22 | 23 | ```scala 24 | requiredChildDistribution: Seq[Distribution] 25 | ``` 26 | 27 | `requiredChildDistribution` is part of the `SparkPlan` ([Spark SQL]({{ book.spark_sql }}/physical-operators/SparkPlan/#requiredChildDistribution)) abstraction. 28 | 29 | --- 30 | 31 | `requiredChildDistribution`...FIXME 32 | -------------------------------------------------------------------------------- /docs/physical-operators/StateStoreReader.md: -------------------------------------------------------------------------------- 1 | == [[StateStoreReader]] StateStoreReader 2 | 3 | `StateStoreReader` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/physical-operators/StatefulOperator.md: -------------------------------------------------------------------------------- 1 | # StatefulOperator Physical Operators 2 | 3 | `StatefulOperator` is the <> of <> that <> or <> state (described by <>). 4 | 5 | [[contract]] 6 | .StatefulOperator Contract 7 | [cols="1m,2",options="header",width="100%"] 8 | |=== 9 | | Method 10 | | Description 11 | 12 | | stateInfo 13 | a| [[stateInfo]] 14 | 15 | [source, scala] 16 | ---- 17 | stateInfo: Option[StatefulOperatorStateInfo] 18 | ---- 19 | 20 | The [StatefulOperatorStateInfo](../stateful-stream-processing/StatefulOperatorStateInfo.md) of the physical operator 21 | |=== 22 | 23 | [[extensions]] 24 | .StatefulOperators (Direct Implementations) 25 | [cols="1,2",options="header",width="100%"] 26 | |=== 27 | | StatefulOperator 28 | | Description 29 | 30 | | <> 31 | | [[StateStoreReader]] 32 | 33 | | [StateStoreWriter](StateStoreWriter.md) 34 | | [[StateStoreWriter]] Physical operator that writes to a state store and collects the write metrics for execution progress reporting 35 | |=== 36 | -------------------------------------------------------------------------------- /docs/physical-operators/StreamingRelationExec.md: -------------------------------------------------------------------------------- 1 | # StreamingRelationExec Leaf Physical Operator 2 | 3 | `StreamingRelationExec` is a leaf physical operator (i.e. `LeafExecNode`) that...FIXME 4 | 5 | ## Creating Instance 6 | 7 | `StreamingRelationExec` takes the following when created: 8 | 9 | * [[sourceName]] The name of a [streaming source](../Source.md) 10 | * [[output]] Output attributes 11 | 12 | `StreamingRelationExec` is created when [StreamingRelationStrategy](../execution-planning-strategies/StreamingRelationStrategy.md) execution planning strategy is executed (to plan `StreamingRelation` and `StreamingExecutionRelation` logical operators). 13 | 14 | ## Demo 15 | 16 | ```text 17 | scala> spark.version 18 | res0: String = 2.3.0-SNAPSHOT 19 | 20 | val rates = spark. 21 | readStream. 22 | format("rate"). 23 | load 24 | 25 | // StreamingRelation logical operator 26 | scala> println(rates.queryExecution.logical.numberedTreeString) 27 | 00 StreamingRelation DataSource(org.apache.spark.sql.SparkSession@31ba0af0,rate,List(),None,List(),None,Map(),None), rate, [timestamp#0, value#1L] 28 | 29 | // StreamingRelationExec physical operator (shown without "Exec" suffix) 30 | scala> rates.explain 31 | == Physical Plan == 32 | StreamingRelation rate, [timestamp#0, value#1L] 33 | ``` 34 | -------------------------------------------------------------------------------- /docs/physical-operators/WriteToDataSourceV2Exec.md: -------------------------------------------------------------------------------- 1 | # WriteToDataSourceV2Exec Physical Operator 2 | 3 | `WriteToDataSourceV2Exec` is a `V2TableWriteExec` ([Spark SQL]({{ book.spark_sql }}/physical-operators/V2TableWriteExec)) that represents [WriteToDataSourceV2](../logical-operators/WriteToDataSourceV2.md) logical operator at execution time. 4 | 5 | ## Creating Instance 6 | 7 | `WriteToDataSourceV2Exec` takes the following to be created: 8 | 9 | * `BatchWrite` ([Spark SQL]({{ book.spark_sql }}/connector/BatchWrite)) 10 | * Refresh Cache Function (`() => Unit`) 11 | * Physical Query Plan ([Spark SQL]({{ book.spark_sql }}/physical-operators/SparkPlan)) 12 | * Write `CustomMetric`s ([Spark SQL]({{ book.spark_sql }}/connector/CustomMetric)) 13 | 14 | `WriteToDataSourceV2Exec` is created when: 15 | 16 | * `DataSourceV2Strategy` ([Spark SQL]({{ book.spark_sql }}/execution-planning-strategies/DataSourceV2Strategy)) execution planning strategy is requested to plan a logical query plan (that is a [WriteToDataSourceV2](../logical-operators/WriteToDataSourceV2.md) logical operator) 17 | 18 | ## Executing Physical Operator 19 | 20 | ```scala 21 | run(): Seq[InternalRow] 22 | ``` 23 | 24 | `run` is part of the `V2CommandExec` ([Spark SQL]({{ book.spark_sql }}/physical-operators/V2TableWriteExec#run)) abstraction. 25 | 26 | --- 27 | 28 | `run` writes rows out ([Spark SQL]({{ book.spark_sql }}/physical-operators/V2TableWriteExec#writeWithV2)) using the [BatchWrite](#batchWrite) and then refreshes the cache (using the [refresh cache function](#refreshCache)). 29 | 30 | In the end, `run` returns the rows written out. 31 | 32 | ## Logging 33 | 34 | Enable `ALL` logging level for `org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec` logger to see what happens inside. 35 | 36 | Add the following line to `conf/log4j.properties`: 37 | 38 | ```text 39 | log4j.logger.org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec=ALL 40 | ``` 41 | 42 | Refer to [Logging](../spark-logging.md). 43 | -------------------------------------------------------------------------------- /docs/rocksdb/index.md: -------------------------------------------------------------------------------- 1 | # RocksDB State Store 2 | 3 | [RocksDB](https://rocksdb.org/) can be used as a state store backend in Spark Structured Streaming. 4 | 5 | !!! quote "RocksDB's Notable Features" 6 | 7 | RocksDB is an embeddable persistent key-value store with the following features: 8 | 9 | * Uses a log structured database engine 10 | * Keys and values are arbitrarily-sized byte streams 11 | * Optimized for fast, low latency storage (flash drives and high-speed disk drives) for high read/write rates 12 | 13 | The full documentation is currently on the [GitHub wiki](https://github.com/facebook/rocksdb/wiki). 14 | 15 | ## stateStore.providerClass 16 | 17 | [spark.sql.streaming.stateStore.providerClass](../configuration-properties.md#spark.sql.streaming.stateStore.providerClass) with `org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider` enables [RocksDBStateStoreProvider](RocksDBStateStoreProvider.md) as the default [StateStoreProvider](../stateful-stream-processing/StateStoreProvider.md). 18 | 19 | ## Logging 20 | 21 | `RocksDB` is used to [create a native logger](RocksDB.md#createLogger) and configure a logging level accordingly. 22 | 23 | ## Demo 24 | 25 | * [Demo: RocksDB State Store for Streaming Aggregation](../demo/rocksdb-state-store-for-streaming-aggregation.md) 26 | -------------------------------------------------------------------------------- /docs/spark-logging.md: -------------------------------------------------------------------------------- 1 | # Logging 2 | 3 | !!! tip 4 | Find out more on logging in Apache Spark in [The Internals of Apache Spark]({{ book.spark_core }}/spark-logging) online book. 5 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/BaseStateStoreRDD.md: -------------------------------------------------------------------------------- 1 | # BaseStateStoreRDD 2 | 3 | `BaseStateStoreRDD` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/ReadStateStore.md: -------------------------------------------------------------------------------- 1 | # ReadStateStore 2 | 3 | `ReadStateStore` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/ReadStateStoreRDD.md: -------------------------------------------------------------------------------- 1 | # ReadStateStoreRDD 2 | 3 | `ReadStateStoreRDD` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StateSchemaCompatibilityChecker.md: -------------------------------------------------------------------------------- 1 | # StateSchemaCompatibilityChecker 2 | 3 | `StateSchemaCompatibilityChecker` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StateStoreConf.md: -------------------------------------------------------------------------------- 1 | # StateStoreConf 2 | 3 | ## minDeltasForSnapshot 4 | 5 | [spark.sql.streaming.stateStore.minDeltasForSnapshot](../configuration-properties.md#spark.sql.streaming.stateStore.minDeltasForSnapshot) 6 | 7 | ## maxVersionsToRetainInMemory 8 | 9 | [spark.sql.streaming.maxBatchesToRetainInMemory](../configuration-properties.md#spark.sql.streaming.maxBatchesToRetainInMemory) 10 | 11 | ## minVersionsToRetain 12 | 13 | [spark.sql.streaming.minBatchesToRetain](../configuration-properties.md#spark.sql.streaming.minBatchesToRetain) 14 | 15 | Used when `HDFSBackedStateStoreProvider` is requested for [cleanup](HDFSBackedStateStoreProvider.md#cleanup). 16 | 17 | ## providerClass 18 | 19 | [spark.sql.streaming.stateStore.providerClass](../configuration-properties.md#spark.sql.streaming.stateStore.providerClass) 20 | 21 | Used when `StateStoreProvider` helper object is requested to [create and initialize the StateStoreProvider](StateStoreProvider.md#createAndInit). 22 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StateStoreCustomMetric.md: -------------------------------------------------------------------------------- 1 | # StateStoreCustomMetric 2 | 3 | `StateStoreCustomMetric` is the <> of <> that a state store may wish to expose (as [StateStoreMetrics](StateStoreMetrics.md) or <>). 4 | 5 | `StateStoreCustomMetric` is used when: 6 | 7 | * `StateStoreProvider` is requested for the <> 8 | 9 | * `StateStoreMetrics` is [created](StateStoreMetrics.md#customMetrics) 10 | 11 | [[contract]] 12 | .StateStoreCustomMetric Contract 13 | [cols="1m,2",options="header",width="100%"] 14 | |=== 15 | | Method 16 | | Description 17 | 18 | | desc 19 | a| [[desc]] 20 | 21 | [source, scala] 22 | ---- 23 | desc: String 24 | ---- 25 | 26 | Description of the custom metrics 27 | 28 | | name 29 | a| [[name]] 30 | 31 | [source, scala] 32 | ---- 33 | name: String 34 | ---- 35 | 36 | Name of the custom metrics 37 | 38 | |=== 39 | 40 | [[implementations]] 41 | .StateStoreCustomMetrics 42 | [cols="1m,2",options="header",width="100%"] 43 | |=== 44 | | StateStoreCustomMetric 45 | | Description 46 | 47 | | StateStoreCustomSizeMetric 48 | | [[StateStoreCustomSizeMetric]] 49 | 50 | | StateStoreCustomSumMetric 51 | | [[StateStoreCustomSumMetric]] 52 | 53 | | StateStoreCustomTimingMetric 54 | | [[StateStoreCustomTimingMetric]] 55 | |=== 56 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StateStoreId.md: -------------------------------------------------------------------------------- 1 | # StateStoreId 2 | 3 | [[creating-instance]] 4 | `StateStoreId` is a unique identifier of a [StateStore](StateStore.md) with the following attributes: 5 | 6 | * [[checkpointRootLocation]] *Checkpoint Root Location* - the root directory for state checkpointing 7 | * [[operatorId]] *Operator ID* - a unique ID of the stateful operator 8 | * [[partitionId]] *Partition ID* - the index of the partition 9 | * [[storeName]] *Store Name* - the name of the [state store](StateStore.md) (default: <>) 10 | 11 | `StateStoreId` is <> when: 12 | 13 | * `StateStoreRDD` is requested for the [preferred locations of a partition](StateStoreRDD.md#getPreferredLocations) (executed on the driver) and to [compute it](StateStoreRDD.md#compute) (later on an executor) 14 | 15 | * `StateStoreProviderId` helper object is requested to create a <> (with a <> and the run ID of a streaming query) that is then used for the [preferred locations of a partition](../join/StateStoreAwareZipPartitionsRDD.md#getPreferredLocations) of a `StateStoreAwareZipPartitionsRDD` (executed on the driver) and to...FIXME 16 | 17 | [[DEFAULT_STORE_NAME]] 18 | The name of the *default state store* (for reading state store data that was generated before store names were used, i.e. in Spark 2.2 and earlier) is *default*. 19 | 20 | === [[storeCheckpointLocation]] State Checkpoint Base Directory of Stateful Operator -- `storeCheckpointLocation` Method 21 | 22 | [source, scala] 23 | ---- 24 | storeCheckpointLocation(): Path 25 | ---- 26 | 27 | `storeCheckpointLocation` is Hadoop DFS's https://hadoop.apache.org/docs/r2.7.3/api/org/apache/hadoop/fs/Path.html[Path] of the checkpoint location (for the stateful operator by <>, the partition by the <> in the <>). 28 | 29 | If the <> is used (for Spark 2.2 and earlier), the <> is not included in the path. 30 | 31 | `storeCheckpointLocation` is used when `HDFSBackedStateStoreProvider` is requested for the [state checkpoint base directory](HDFSBackedStateStoreProvider.md#baseDir). 32 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StateStoreMetrics.md: -------------------------------------------------------------------------------- 1 | # StateStoreMetrics 2 | 3 | `StateStoreMetrics` holds the performance metrics of [StateStore](StateStore.md#metrics)s and [SymmetricHashJoinStateManager](../join/SymmetricHashJoinStateManager.md). 4 | 5 | ## Creating Instance 6 | 7 | `StateStoreMetrics` takes the following to be created: 8 | 9 | * Number of Keys 10 | * Memory used (in bytes) 11 | * [Custom Metrics](#customMetrics) 12 | 13 | `StateStoreMetrics` is created when: 14 | 15 | * `HDFSBackedStateStore` is requested for [metrics](HDFSBackedStateStore.md#metrics) 16 | * `RocksDBStateStore` is requested for [metrics](../rocksdb/RocksDBStateStore.md#metrics) 17 | * `StateStoreMetrics` is requested to [combine metrics](#combine) 18 | * `SymmetricHashJoinStateManager` is requested for [metrics](../join/SymmetricHashJoinStateManager.md#metrics) 19 | 20 | ### Custom Metrics 21 | 22 | `StateStoreMetrics` is given [StateStoreCustomMetric](StateStoreCustomMetric.md)s and their current values when [created](#creating-instance). 23 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StatefulOperatorPartitioning.md: -------------------------------------------------------------------------------- 1 | # StatefulOperatorPartitioning 2 | 3 | ## getCompatibleDistribution 4 | 5 | ```scala 6 | getCompatibleDistribution( 7 | expressions: Seq[Expression], 8 | numPartitions: Int, 9 | conf: SQLConf): Distribution 10 | getCompatibleDistribution( 11 | expressions: Seq[Expression], 12 | stateInfo: StatefulOperatorStateInfo, 13 | conf: SQLConf): Distribution 14 | ``` 15 | 16 | `getCompatibleDistribution` returns the following `Distribution`s ([Spark SQL]({{ book.spark_sql }}/physical-operators/Distribution/)) based on [spark.sql.streaming.statefulOperator.useStrictDistribution](../configuration-properties.md#STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION) configuration property: 17 | 18 | * `StatefulOpClusteredDistribution` when enabled 19 | * `ClusteredDistribution` ([Spark SQL]({{ book.spark_sql }}/physical-operators/ClusteredDistribution)) otherwise 20 | 21 | --- 22 | 23 | `getCompatibleDistribution` is used when: 24 | 25 | * `BaseAggregateExec` ([Spark SQL]({{ book.spark_sql }}/physical-operators/BaseAggregateExec/)) is requested for the required child output distribution (of a streaming query) 26 | * `UpdatingSessionsExec` is requested for the required child output distribution (of a streaming query) 27 | * `FlatMapGroupsWithStateExec` is requested for the [required child output distribution](../physical-operators/FlatMapGroupsWithStateExec.md#requiredChildDistribution) 28 | * `StateStoreRestoreExec` is requested for the [required child output distribution](../physical-operators/StateStoreRestoreExec.md#requiredChildDistribution) 29 | * `StateStoreSaveExec` is requested for the [required child output distribution](../physical-operators/StateStoreSaveExec.md#requiredChildDistribution) 30 | * `SessionWindowStateStoreRestoreExec` is requested for the [required child output distribution](../physical-operators/SessionWindowStateStoreRestoreExec.md#requiredChildDistribution) 31 | * `SessionWindowStateStoreSaveExec` is requested for the [required child output distribution](../physical-operators/SessionWindowStateStoreSaveExec.md#requiredChildDistribution) 32 | * `StreamingDeduplicateExec` is requested for the [required child output distribution](../physical-operators/StreamingDeduplicateExec.md#requiredChildDistribution) 33 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StatefulOperatorStateInfo.md: -------------------------------------------------------------------------------- 1 | # StatefulOperatorStateInfo 2 | 3 | `StatefulOperatorStateInfo` identifies the state store for a given stateful physical operator: 4 | 5 | * [[checkpointLocation]] Checkpoint directory (`checkpointLocation`) 6 | * [[queryRunId]] <> of a streaming query (`queryRunId`) 7 | * [[operatorId]] Stateful operator ID (`operatorId`) 8 | * [[storeVersion]] <> (`storeVersion`) 9 | * [[numPartitions]] Number of partitions 10 | 11 | `StatefulOperatorStateInfo` is <> exclusively when `IncrementalExecution` is requested for [nextStatefulOperationStateInfo](../IncrementalExecution.md#nextStatefulOperationStateInfo). 12 | 13 | [[toString]] 14 | When requested for a textual representation (`toString`), `StatefulOperatorStateInfo` returns the following: 15 | 16 | ```text 17 | state info [ checkpoint = [checkpointLocation], runId = [queryRunId], opId = [operatorId], ver = [storeVersion], numPartitions = [numPartitions]] 18 | ``` 19 | 20 | ## State Version and Batch ID 21 | 22 | When created (when `IncrementalExecution` is requested for the [next StatefulOperatorStateInfo](../IncrementalExecution.md#nextStatefulOperationStateInfo)), a `StatefulOperatorStateInfo` is given a [state version](#storeVersion). 23 | 24 | The state version is exactly the [batch ID](../IncrementalExecution.md#currentBatchId) of the [IncrementalExecution](../IncrementalExecution.md). 25 | -------------------------------------------------------------------------------- /docs/stateful-stream-processing/StreamingSessionWindowStateManager.md: -------------------------------------------------------------------------------- 1 | # StreamingSessionWindowStateManager 2 | 3 | `StreamingSessionWindowStateManager` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/streaming-aggregation/StreamingAggregationStateManagerBaseImpl.md: -------------------------------------------------------------------------------- 1 | # StreamingAggregationStateManagerBaseImpl 2 | 3 | `StreamingAggregationStateManagerBaseImpl` is a base implementation of the [StreamingAggregationStateManager](StreamingAggregationStateManager.md) abstraction for [state managers for streaming aggregations](#implementations). 4 | 5 | ## Implementations 6 | 7 | * `StreamingAggregationStateManagerImplV1` (legacy) 8 | * [StreamingAggregationStateManagerImplV2](StreamingAggregationStateManagerImplV2.md) 9 | 10 | ## Creating Instance 11 | 12 | `StreamingAggregationStateManagerBaseImpl` takes the following to be created: 13 | 14 | * Key `Attribute`s ([Spark SQL]({{ book.spark_sql }}/expressions/Attribute)) 15 | * Input Row `Attribute`s ([Spark SQL]({{ book.spark_sql }}/expressions/Attribute)) 16 | 17 | !!! note "Abstract Class" 18 | `StreamingAggregationStateManagerBaseImpl` is an abstract class and cannot be created directly. It is created indirectly for the [concrete StreamingAggregationStateManagerBaseImpls](#implementations). 19 | 20 | ## Committing State Changes 21 | 22 | ```scala 23 | commit( 24 | store: StateStore): Long 25 | ``` 26 | 27 | `commit` is part of the [StreamingAggregationStateManager](StreamingAggregationStateManager.md#commit) abstraction. 28 | 29 | --- 30 | 31 | `commit` requests the given [StateStore](../stateful-stream-processing/StateStore.md) to [commit state changes](../stateful-stream-processing/StateStore.md#commit). 32 | 33 | ## Extracting Key 34 | 35 | ```scala 36 | getKey( 37 | row: UnsafeRow): UnsafeRow 38 | ``` 39 | 40 | `getKey` is part of the [StreamingAggregationStateManager](StreamingAggregationStateManager.md#getKey) abstraction. 41 | 42 | --- 43 | 44 | `getKey` uses the [keyProjector](#keyProjector) to extract a key from the given `row`. 45 | 46 | ## Removing Key 47 | 48 | ```scala 49 | remove( 50 | store: StateStore, 51 | key: UnsafeRow): Unit 52 | ``` 53 | 54 | `remove` is part of the [StreamingAggregationStateManager](StreamingAggregationStateManager.md#remove) abstraction. 55 | 56 | --- 57 | 58 | `remove` requests the given [StateStore](../stateful-stream-processing/StateStore.md) to [remove](../stateful-stream-processing/StateStore.md#remove) the given `key`. 59 | -------------------------------------------------------------------------------- /docs/streaming-deduplication/index.md: -------------------------------------------------------------------------------- 1 | == Streaming Deduplication 2 | 3 | *Streaming Deduplication* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/streaming-limit/index.md: -------------------------------------------------------------------------------- 1 | == Streaming Limit 2 | 3 | *Streaming Limit* is...FIXME 4 | -------------------------------------------------------------------------------- /docs/testing.md: -------------------------------------------------------------------------------- 1 | # Testing Streaming Queries 2 | 3 | Spark Structured Streaming comes with the built-in [Memory Data Source](datasources/memory/index.md) for writing tests of streaming queries. 4 | 5 | The idea is to load data from a [memory source](datasources/memory/MemoryStreamBase.md), execute data stream transformations (your code), and write the result out to a [memory sink](datasources/memory/MemorySink.md) (that becomes a queryable temporary view). 6 | 7 | ## Demo 8 | 9 | !!! tip 10 | Review [MemorySinkSuite]({{ spark.github }}/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala#L31) and [FileStreamSourceSuite]({{ spark.github }}/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala#L227) test suites in the source code of Apache Spark. 11 | 12 | ## Learn More 13 | 14 | * [How to perform Unit testing on Spark Structured Streaming?](https://stackoverflow.com/q/56894068/1305344) 15 | * [Unit Testing Spark Structured Streaming Application using Memory Stream](https://blog.devgenius.io/unit-testing-spark-structured-streaming-application-using-memory-stream-fbaebfd39791) 16 | * [Integration tests and Structured Streaming](https://www.waitingforcode.com/apache-spark-structured-streaming/integration-tests-structured-streaming/read) 17 | * [Writing Unit Test for Apache Spark using Memory Streams](https://blog.knoldus.com/apache-sparks-memory-streams/) 18 | -------------------------------------------------------------------------------- /docs/watermark/EventTimeStats.md: -------------------------------------------------------------------------------- 1 | # EventTimeStats 2 | 3 | `EventTimeStats` is used to help calculate **event-time column statistics** (statistics of the values of an event-time column): 4 | 5 | * Maximum 6 | * Minimum 7 | * Average 8 | * Count 9 | 10 | `EventTimeStats` is used by [EventTimeStatsAccum](EventTimeStatsAccum.md) accumulator. 11 | 12 | ## Zero Value 13 | 14 | `EventTimeStats` defines a special value **zero** with the following values: 15 | 16 | * `Long.MinValue` for the [max](#max) 17 | * `Long.MaxValue` for the [min](#min) 18 | * `0.0` for the [avg](#avg) 19 | * `0L` for the [count](#count) 20 | 21 | ## Adding Event-Time Value 22 | 23 | ```scala 24 | add( 25 | eventTime: Long): Unit 26 | ``` 27 | 28 | `add` updates the statistics given the `eventTime` value. 29 | 30 | ## Merging EventTimeStats 31 | 32 | ```scala 33 | merge( 34 | that: EventTimeStats): Unit 35 | ``` 36 | 37 | `merge`...FIXME 38 | -------------------------------------------------------------------------------- /docs/watermark/EventTimeStatsAccum.md: -------------------------------------------------------------------------------- 1 | # EventTimeStatsAccum Accumulator 2 | 3 | `EventTimeStatsAccum` is an `AccumulatorV2` ([Spark Core]({{ book.spark_core }}/accumulators/AccumulatorV2/)) that accumulates `Long` values and produces an [EventTimeStats](EventTimeStats.md). 4 | 5 | ```scala 6 | AccumulatorV2[Long, EventTimeStats] 7 | ``` 8 | 9 | ## Creating Instance 10 | 11 | `EventTimeStatsAccum` takes the following to be created: 12 | 13 | * [EventTimeStats](#currentStats) (default: [EventTimeStats.zero](EventTimeStats.md#zero)) 14 | 15 | `EventTimeStatsAccum` is created when: 16 | 17 | * [EventTimeWatermarkExec](../physical-operators/EventTimeWatermarkExec.md) unary physical operator is created (and initializes [eventTimeStats](../physical-operators/EventTimeWatermarkExec.md#eventTimeStats)) 18 | 19 | !!! note "EventTimeWatermarkExec Physical Operator" 20 | When `EventTimeWatermarkExec` physical operator is requested to execute, every task simply [adds](#add) the values of the [event-time watermark column expression](../physical-operators/EventTimeWatermarkExec.md#eventTime) to the [EventTimeStatsAccum](../physical-operators/EventTimeWatermarkExec.md#eventTimeStats) accumulator. 21 | 22 | As per design of Spark accumulators in Apache Spark, accumulator updates are automatically sent out (_propagated_) from tasks to the driver every heartbeat and then they are accumulated together. 23 | 24 | ## EventTimeStats 25 | 26 | `EventTimeStatsAccum` is given an [EventTimeStats](EventTimeStats.md) when [created](#creating-instance). 27 | 28 | Every time `AccumulatorV2` methods are called, `EventTimeStatsAccum` simply delegates them to the `EventTimeStats` (that is responsible for event-time statistics, i.e. max, min, avg, count). 29 | 30 | ## Adding Value 31 | 32 | ```scala 33 | add( 34 | v: Long): Unit 35 | ``` 36 | 37 | `add` is part of the `AccumulatorV2` ([Spark Core]({{ book.spark_core }}/accumulators/AccumulatorV2/#add)) abstraction. 38 | 39 | `add` simply requests the [EventTimeStats](#currentStats) to [add](EventTimeStats.md#add) the given `v` value. 40 | 41 | `add` is used when [EventTimeWatermarkExec](../physical-operators/EventTimeWatermarkExec.md) physical operator is executed. 42 | -------------------------------------------------------------------------------- /docs/webui/StreamingQueryStatusStore.md: -------------------------------------------------------------------------------- 1 | # StreamingQueryStatusStore 2 | 3 | ## Creating Instance 4 | 5 | `StreamingQueryStatusStore` takes the following to be created: 6 | 7 | * `KVStore` ([Spark Core]({{ book.spark_core }}/core/KVStore)) 8 | 9 | `StreamingQueryStatusStore` is created when: 10 | 11 | * `StreamingQueryHistoryServerPlugin` is requested to `setupUI` 12 | * `SharedState` ([Spark SQL]({{ book.spark_sql }}/SharedState)) is created (with [spark.sql.streaming.ui.enabled](../configuration-properties.md#spark.sql.streaming.ui.enabled) enabled) 13 | 14 | ## allQueryUIData 15 | 16 | ```scala 17 | allQueryUIData: Seq[StreamingQueryUIData] 18 | ``` 19 | 20 | `allQueryUIData` creates a view of `StreamingQueryData`s (in the [KVStore](#store)) indexed by `startTimestamp` to [makeUIData](#makeUIData). 21 | 22 | --- 23 | 24 | `allQueryUIData` is used when: 25 | 26 | * `StreamingQueryHistoryServerPlugin` is requested to `setupUI` 27 | * `StreamingQueryPage` is requested to [generateStreamingQueryTable](StreamingQueryPage.md#generateStreamingQueryTable) 28 | * `StreamingQueryStatisticsPage` is requested to [render](StreamingQueryStatisticsPage.md#render) 29 | 30 | ### makeUIData 31 | 32 | ```scala 33 | makeUIData( 34 | summary: StreamingQueryData): StreamingQueryUIData 35 | ``` 36 | 37 | `makeUIData`...FIXME 38 | -------------------------------------------------------------------------------- /docs/webui/StreamingQueryTab.md: -------------------------------------------------------------------------------- 1 | # StreamingQueryTab 2 | 3 | `StreamingQueryTab` is a `SparkUITab` ([Spark Core]({{ book.spark_core }}/webui/SparkUITab)) with `StreamingQuery` URL prefix. 4 | 5 | When [created](#creating-instance), `StreamingQueryTab` attaches the following pages: 6 | 7 | * [StreamingQueryPage](StreamingQueryPage.md) 8 | * [StreamingQueryStatisticsPage](StreamingQueryStatisticsPage.md) 9 | 10 | `StreamingQueryTab` can be turned on/off using [spark.sql.streaming.ui.enabled](../configuration-properties.md#spark.sql.streaming.ui.enabled). 11 | 12 | ## Creating Instance 13 | 14 | `StreamingQueryTab` takes the following to be created: 15 | 16 | * [StreamingQueryStatusStore](#store) 17 | * `SparkUI` ([Spark Core]({{ book.spark_core }}/webui/SparkUI)) 18 | 19 | `StreamingQueryTab` is created when: 20 | 21 | * `StreamingQueryHistoryServerPlugin` is requested to `setupUI` 22 | * `SharedState` ([Spark SQL]({{ book.spark_sql }}/SharedState)) is created (with [spark.sql.streaming.ui.enabled](../configuration-properties.md#spark.sql.streaming.ui.enabled) enabled) 23 | 24 | ### StreamingQueryStatusStore 25 | 26 | `StreamingQueryTab` is given a [StreamingQueryStatusStore](StreamingQueryStatusStore.md) when [created](#creating-instance). 27 | 28 | The `StreamingQueryStatusStore` is used to [fetch the streaming query data](StreamingQueryStatusStore.md#allQueryUIData) in the attached pages: 29 | 30 | * [StreamingQueryPage](StreamingQueryPage.md#generateStreamingQueryTable) 31 | * [StreamingQueryStatisticsPage](StreamingQueryStatisticsPage.md#render) 32 | 33 | ## Tab Name 34 | 35 | ```scala 36 | name: String 37 | ``` 38 | 39 | `name` is part of the `WebUITab` ([Spark Core]({{ book.spark_core }}/webui/WebUITab/#name)) abstraction. 40 | 41 | --- 42 | 43 | `name` is `Structured Streaming`. 44 | -------------------------------------------------------------------------------- /docs/webui/index.md: -------------------------------------------------------------------------------- 1 | # Structured Streaming UI 2 | 3 | Structured Streaming applications can be monitored using [web UI](StreamingQueryTab.md) that attaches the following two pages: 4 | 5 | * [Streaming Query](StreamingQueryPage.md) 6 | * [Streaming Query Statistics](StreamingQueryStatisticsPage.md) 7 | 8 | Streaming events are intercepted using [StreamingQueryStatusListener](StreamingQueryStatusListener.md) (and persisted in the [store](StreamingQueryStatusListener.md#store)). The store is used to create a [StreamingQueryStatusStore](StreamingQueryStatusStore.md#store) for [StreamingQueryTab](StreamingQueryTab.md#store) (and the pages). 9 | -------------------------------------------------------------------------------- /examples/.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | project 3 | target -------------------------------------------------------------------------------- /examples/README.md: -------------------------------------------------------------------------------- 1 | # Spark Structured Streaming Examples 2 | 3 | ## Introduction 4 | 5 | The project contains complete Spark applications (using Scala and sbt) to demonstrate various features of Spark Structured Streaming: 6 | 7 | * [FlatMapGroupsWithStateApp](src/main/scala/pl/japila/spark/FlatMapGroupsWithStateApp.scala) 8 | * [GroupbyAppendWatermarkExample](src/main/scala/pl/japila/spark/GroupbyAppendWatermarkExample.scala) 9 | * [StreamingAggregationApp](src/main/scala/pl/japila/spark/StreamingAggregationApp.scala) 10 | * [StreamStreamJoinApp](src/main/scala/pl/japila/spark/StreamStreamJoinApp.scala) 11 | 12 | **TIP**: Read up on the features in [The Internals of Spark Structured Streaming](https://bit.ly/spark-structured-streaming) online book. 13 | 14 | ## Getting Started 15 | 16 | Just browse [src/main/scala](src/main/scala/pl/japila/spark) directory and read the scaladoc of the example of your interest. 17 | -------------------------------------------------------------------------------- /examples/build.sbt: -------------------------------------------------------------------------------- 1 | name := "spark-structured-streaming-examples" 2 | 3 | ThisBuild / organization := "pl.japila" 4 | ThisBuild / version := "0.1" 5 | ThisBuild / scalaVersion := "2.12.8" 6 | 7 | val sparkVersion = "2.4.4" 8 | lazy val commonSettings = Seq( 9 | libraryDependencies += "org.apache.spark" %% "spark-sql" % sparkVersion 10 | ) 11 | 12 | lazy val p0 = (project in file(".")) 13 | .aggregate(p1) 14 | .settings(commonSettings) 15 | 16 | lazy val p1 = (project in file("streaming-source")) 17 | .settings( 18 | commonSettings 19 | ) 20 | -------------------------------------------------------------------------------- /examples/project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 1.3.3 -------------------------------------------------------------------------------- /examples/src/main/resources/KafkaSourceExample.scala: -------------------------------------------------------------------------------- 1 | /** 2 | ./bin/kafka-console-producer.sh \ 3 | --topic topic1 \ 4 | --broker-list localhost:9092 \ 5 | --property parse.key=true \ 6 | --property key.separator=, 7 | */ 8 | import org.apache.spark.sql.streaming.{OutputMode, Trigger} 9 | import scala.concurrent.duration._ 10 | val fromKafkaTopic1ToConsole = spark. 11 | readStream. 12 | format("kafka"). 13 | option("subscribe", "topic1"). 14 | option("kafka.bootstrap.servers", "localhost:9092"). 15 | option("startingOffsets", "earliest"). // latest, earliest or JSON with {"topicA":{"part":offset,"p1":-1},"topicB":{"0":-2}} 16 | load. 17 | select($"key" cast "string", $"value" cast "string"). // deserialize records 18 | as[(String, String)]. 19 | writeStream. 20 | trigger(Trigger.ProcessingTime(10.seconds)). 21 | queryName("from-kafka-to-console"). 22 | outputMode(OutputMode.Append). 23 | format("console"). 24 | start 25 | 26 | // ...after some time 27 | fromKafkaTopic1ToConsole.stop 28 | -------------------------------------------------------------------------------- /examples/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=OFF, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 6 | 7 | log4j.logger.org.apache.spark.sql.execution.streaming.WatermarkTracker=ALL 8 | log4j.logger.org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider=ALL 9 | log4j.logger.org.apache.spark.sql.execution.streaming.MemorySink=ALL 10 | log4j.logger.org.apache.spark.sql.execution.streaming.MemoryStream=ALL 11 | 12 | # Stream-Stream Joins 13 | log4j.logger.org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys=ALL 14 | log4j.logger.org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper=ALL 15 | log4j.logger.org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager$KeyToNumValuesStore=ALL 16 | log4j.logger.org.apache.spark.sql.execution.streaming.state.SymmetricHashJoinStateManager$KeyWithIndexToValueStore=ALL -------------------------------------------------------------------------------- /examples/src/main/scala/pl/japila/spark/Event.scala: -------------------------------------------------------------------------------- 1 | package pl.japila.spark 2 | 3 | import java.sql.Timestamp 4 | 5 | // Define event "format" 6 | // Event time must be defined on a window or a timestamp 7 | case class Event(time: Timestamp, value: Long, batch: Long) 8 | 9 | import scala.concurrent.duration._ 10 | object Event { 11 | def apply(secs: Long, value: Long, batch: Long): Event = { 12 | Event(new Timestamp(secs.seconds.toMillis), value, batch) 13 | } 14 | 15 | def apply(value: Long, batch: Long): Event = { 16 | Event(batch * 10, value, batch) 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /examples/src/main/scala/pl/japila/spark/GroupbyAppendWatermarkExample.scala: -------------------------------------------------------------------------------- 1 | package pl.japila.spark 2 | 3 | /** 4 | * Consumes lines from a socket (in the format of "second,group") 5 | * groups them by group and prints the aggregations to the console 6 | * 7 | * Use `nc -l 8888` to open a socket 8 | */ 9 | object GroupbyAppendWatermarkExample extends SparkStreamsApp { 10 | 11 | val source = "socket" 12 | val sink = "console" 13 | 14 | val host = "localhost" 15 | val port = "8888" 16 | 17 | val eventTimeCol = "eventTime" 18 | 19 | import spark.implicits._ 20 | import org.apache.spark.sql.DataFrame 21 | val mytransform: DataFrame => DataFrame = { df => 22 | import org.apache.spark.sql.functions._ 23 | df 24 | .as[String] 25 | .map { line => 26 | val ts = line.split(",") 27 | (ts(0), ts(1)) 28 | } 29 | .toDF(eventTimeCol, "value") 30 | .withColumn(eventTimeCol, col(eventTimeCol) cast "long" cast "timestamp") 31 | .withWatermark(eventTimeCol, "0 seconds") 32 | .groupBy($"value", window(col(eventTimeCol), "5 seconds") as "window") 33 | .agg(collect_list(col(eventTimeCol) cast "long") as "times") 34 | .select("value", "times", "window") 35 | } 36 | 37 | val input = spark 38 | .readStream 39 | .format(source) 40 | .option("host", host) 41 | .option("port", port) 42 | .load 43 | 44 | val output = input.transform(mytransform) 45 | 46 | import concurrent.duration._ 47 | import org.apache.spark.sql.streaming.{OutputMode, Trigger} 48 | val sq = output 49 | .writeStream 50 | .format(sink) 51 | .option("truncate", false) 52 | .queryName(queryName) 53 | .option("checkpointLocation", checkpointLocation) 54 | .trigger(Trigger.ProcessingTime(5.seconds)) 55 | .outputMode(OutputMode.Append) 56 | .start 57 | 58 | sq.awaitTermination() 59 | 60 | } 61 | -------------------------------------------------------------------------------- /examples/src/main/scala/pl/japila/spark/SparkStreamsApp.scala: -------------------------------------------------------------------------------- 1 | package pl.japila.spark 2 | 3 | import java.io.{BufferedReader, InputStreamReader} 4 | 5 | /** 6 | * Base of Spark Structured Streaming (aka Spark Streams) applications 7 | */ 8 | trait SparkStreamsApp extends App { 9 | val appName = this.getClass.getSimpleName.replace("$", "") 10 | val queryName = appName 11 | val rootDir = "target" 12 | val checkpointLocation = s"$rootDir/checkpoint-$queryName" 13 | val numPartitions = 1 14 | val master = "local[*]" 15 | val warehouseDir = s"$rootDir/$queryName-warehouse" 16 | 17 | import org.apache.spark.sql.SparkSession 18 | val spark = SparkSession 19 | .builder 20 | .master(master) 21 | .appName(appName) 22 | .config("spark.sql.shuffle.partitions", numPartitions) 23 | .config("spark.sql.warehouse.dir", warehouseDir) 24 | .getOrCreate 25 | 26 | /** 27 | * Deletes the checkpoint location from previous executions 28 | */ 29 | def deleteCheckpointLocation(): Unit = { 30 | println(s">>> Deleting checkpoint location: $checkpointLocation") 31 | import java.nio.file.{Files, FileSystems} 32 | import java.util.Comparator 33 | import scala.collection.JavaConverters._ 34 | val path = FileSystems.getDefault.getPath(checkpointLocation) 35 | if (Files.exists(path)) { 36 | Files.walk(path) 37 | .sorted(Comparator.reverseOrder()) 38 | .iterator 39 | .asScala 40 | .foreach(p => p.toFile.delete) 41 | } 42 | } 43 | 44 | def pause() = { 45 | println("Pause processing") 46 | val webUrl = spark.sparkContext.uiWebUrl.get 47 | println(s"Let's you analyze the logs and web UI @ $webUrl") 48 | println("Press ENTER to continue...") 49 | val input = new BufferedReader(new InputStreamReader(System.in)) 50 | input.readLine() 51 | } 52 | 53 | type Millis = Long 54 | def toMillis(datetime: String): Millis = { 55 | import java.time.format.DateTimeFormatter 56 | import java.time.LocalDateTime 57 | import java.time.ZoneOffset 58 | LocalDateTime 59 | .parse(datetime, DateTimeFormatter.ISO_DATE_TIME) 60 | .toInstant(ZoneOffset.UTC) 61 | .toEpochMilli 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /examples/src/main/scala/pl/japila/spark/StreamBatchJoinApp.scala: -------------------------------------------------------------------------------- 1 | package pl.japila.spark 2 | 3 | object StreamBatchJoinApp extends SparkStreamsApp { 4 | import spark.implicits._ 5 | 6 | val stream = spark.readStream.format("rate").load 7 | 8 | { 9 | val batch = spark.read.format("text").load("target/files") 10 | val joined = stream.join(batch, "value") 11 | } 12 | 13 | { 14 | val batch2 = (0 to 10).toDF("value") 15 | val joined2 = batch2.join(stream, "value") 16 | 17 | joined2.explain 18 | assert(joined2.isStreaming) 19 | joined2.writeStream.format("console").option("truncate", false).start 20 | } 21 | 22 | { 23 | val batch3 = spark.read.format("text").load("target/files") 24 | val joined3 = batch3.join(stream, "value") 25 | 26 | joined3.explain 27 | assert(joined3.isStreaming) 28 | joined3.writeStream.format("console").option("truncate", false).start 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /examples/streaming-source/README.md: -------------------------------------------------------------------------------- 1 | # Demo Streaming Source (Structured Streaming) 2 | 3 | The project contains the sources of a `demo` streaming data source with the following features: 4 | 5 | * Reading (loading) data 6 | * Data Source V1 ([Source](https://jaceklaskowski.gitbooks.io/spark-structured-streaming/spark-sql-streaming-Source.html)) 7 | * [Micro-Batch Stream Processing](https://jaceklaskowski.gitbooks.io/spark-structured-streaming/spark-sql-streaming-micro-batch-stream-processing.html) 8 | 9 | Among the other features are: 10 | 11 | * Resuming execution from `checkpointLocation` 12 | * Restarting query preserves the latest batch and offset processed (committed) 13 | * `SparkContext.runJob` to simulate requesting remote data for `Source.getBatch` 14 | * Use web UI to see the jobs with the same description as the micro-batch (cf. `StreamExecution.getBatchDescriptionString`) 15 | 16 | ## Logging 17 | 18 | The demo uses the custom logging configuration to learn the innerworkings of Structured Streaming and the features from the logs: 19 | 20 | ``` 21 | log4j.logger.org.apache.spark.sql.execution.streaming.MicroBatchExecution=ALL 22 | ``` -------------------------------------------------------------------------------- /examples/streaming-source/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | demo.DemoSourceProvider -------------------------------------------------------------------------------- /examples/streaming-source/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=OFF, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.err 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss} %p %c{1}: %m%n 6 | 7 | log4j.logger.org.apache.spark.sql.execution.streaming.WatermarkTracker=ALL 8 | log4j.logger.org.apache.spark.sql.execution.streaming.MicroBatchExecution=ALL -------------------------------------------------------------------------------- /examples/streaming-source/src/main/scala/demo/DemoSourceProvider.scala: -------------------------------------------------------------------------------- 1 | package demo 2 | 3 | import org.apache.spark.sql.SQLContext 4 | import org.apache.spark.sql.execution.streaming.Source 5 | import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} 6 | import org.apache.spark.sql.types.{LongType, StructField, StructType} 7 | 8 | class DemoSourceProvider 9 | extends StreamSourceProvider 10 | with DataSourceRegister { 11 | 12 | import DemoSourceProvider._ 13 | 14 | override def shortName(): String = SHORT_NAME 15 | 16 | override def sourceSchema( 17 | sqlContext: SQLContext, 18 | schema: Option[StructType], 19 | providerName: String, 20 | parameters: Map[String, String]): (String, StructType) = { 21 | println(s">>> DemoSourceProvider.sourceSchema(" + 22 | s"schema = $schema, " + 23 | s"providerName = $providerName, " + 24 | s"parameters = $parameters)") 25 | val s = schema.getOrElse(DEFAULT_SCHEMA) 26 | val r = (shortName(), s) 27 | println(s"<<< <<< DemoSourceProvider.sourceSchema returns $r") 28 | r 29 | } 30 | 31 | override def createSource( 32 | sqlContext: SQLContext, 33 | metadataPath: String, 34 | schema: Option[StructType], 35 | providerName: String, 36 | parameters: Map[String, String]): Source = new DemoSource(sqlContext) 37 | } 38 | 39 | object DemoSourceProvider { 40 | val SHORT_NAME = "demo" 41 | val DEFAULT_SCHEMA = StructType(StructField("id", LongType) :: Nil) 42 | } 43 | -------------------------------------------------------------------------------- /examples/streaming-source/src/main/scala/demo/Main.scala: -------------------------------------------------------------------------------- 1 | package demo 2 | 3 | import java.util.concurrent.TimeUnit 4 | 5 | import org.apache.spark.sql.SparkSession 6 | import org.apache.spark.sql.streaming.Trigger 7 | 8 | object Main extends App { 9 | 10 | val spark = SparkSession 11 | .builder 12 | .master("local[*]") 13 | .appName("Demo Source (DSv1 / Micro-Batch)") 14 | .getOrCreate 15 | 16 | println(s"This is Spark v${spark.version}") 17 | 18 | val data = spark 19 | .readStream 20 | .format("demo") // <-- DataSourceRegister + META-INF/services 21 | .load 22 | 23 | data.explain 24 | 25 | val stream = data 26 | .writeStream 27 | .queryName("demo") 28 | .format("console") 29 | .trigger(Trigger.ProcessingTime(5, TimeUnit.SECONDS)) 30 | .option("checkpointLocation", "target/checkpointLocation") 31 | .start 32 | stream.awaitTermination() 33 | } 34 | -------------------------------------------------------------------------------- /examples/streaming-source/src/main/scala/org/apache/spark/sql/UsePrivateSqlHack.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql 2 | 3 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 4 | 5 | /** 6 | * Dataset.ofRows is private[sql] 7 | * and we need it to create a streaming DataFrame 8 | */ 9 | object UsePrivateSqlHack { 10 | 11 | def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { 12 | Dataset.ofRows(sparkSession, logicalPlan) 13 | } 14 | 15 | } -------------------------------------------------------------------------------- /graffles/ContinuousExecution-addOffset.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/ContinuousExecution-addOffset.graffle -------------------------------------------------------------------------------- /graffles/DataStreamReader-SparkSession-StreamingRelation.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/DataStreamReader-SparkSession-StreamingRelation.graffle -------------------------------------------------------------------------------- /graffles/KafkaOffsetReader-fetchSpecificOffsets.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/KafkaOffsetReader-fetchSpecificOffsets.graffle -------------------------------------------------------------------------------- /graffles/KafkaSource-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/KafkaSource-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/KafkaSource-initialPartitionOffsets.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/KafkaSource-initialPartitionOffsets.graffle -------------------------------------------------------------------------------- /graffles/KeyToNumValuesStore-KeyWithIndexToValueStore.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/KeyToNumValuesStore-KeyWithIndexToValueStore.graffle -------------------------------------------------------------------------------- /graffles/MicroBatchExecution.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/MicroBatchExecution.graffle -------------------------------------------------------------------------------- /graffles/OneSideHashJoiner.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/OneSideHashJoiner.graffle -------------------------------------------------------------------------------- /graffles/ProgressReporter-finishTrigger-timestamps.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/ProgressReporter-finishTrigger-timestamps.graffle -------------------------------------------------------------------------------- /graffles/ProgressReporter-updateProgress.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/ProgressReporter-updateProgress.graffle -------------------------------------------------------------------------------- /graffles/StateStoreRDD-SparkPlans-LogicalPlans-operators.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreRDD-SparkPlans-LogicalPlans-operators.graffle -------------------------------------------------------------------------------- /graffles/StateStoreRDD-StateStoreCoordinator.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreRDD-StateStoreCoordinator.graffle -------------------------------------------------------------------------------- /graffles/StateStoreRestoreExec-IncrementalExecution.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreRestoreExec-IncrementalExecution.graffle -------------------------------------------------------------------------------- /graffles/StateStoreRestoreExec-StatefulAggregationStrategy.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreRestoreExec-StatefulAggregationStrategy.graffle -------------------------------------------------------------------------------- /graffles/StateStoreSaveExec-IncrementalExecution.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreSaveExec-IncrementalExecution.graffle -------------------------------------------------------------------------------- /graffles/StateStoreSaveExec-StateStoreRDD.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreSaveExec-StateStoreRDD.graffle -------------------------------------------------------------------------------- /graffles/StateStoreSaveExec-StatefulAggregationStrategy.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StateStoreSaveExec-StatefulAggregationStrategy.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-durationMs.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-durationMs.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatch-addBatch.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatch-addBatch.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatch-getBatch.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatch-getBatch.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatch-newBatchesPlan.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatch-newBatchesPlan.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatch-nextBatch.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatch-nextBatch.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatch-queryPlanning.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatch-queryPlanning.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-runBatches.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-runBatches.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-start.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-start.graffle -------------------------------------------------------------------------------- /graffles/StreamExecution-uniqueSources.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamExecution-uniqueSources.graffle -------------------------------------------------------------------------------- /graffles/StreamingDeduplicateExec-StreamingDeduplicationStrategy.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingDeduplicateExec-StreamingDeduplicationStrategy.graffle -------------------------------------------------------------------------------- /graffles/StreamingExecutionRelation.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingExecutionRelation.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryListener-onQueryProgress.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryListener-onQueryProgress.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryListener-onQueryStarted.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryListener-onQueryStarted.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryListener-onQueryTerminated.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryListener-onQueryTerminated.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryListenerBus.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryListenerBus.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryManager-createQuery.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryManager-createQuery.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryManager-notifyQueryTermination.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryManager-notifyQueryTermination.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryManager-postListenerEvent.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryManager-postListenerEvent.graffle -------------------------------------------------------------------------------- /graffles/StreamingQueryManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingQueryManager.graffle -------------------------------------------------------------------------------- /graffles/StreamingRelation.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingRelation.graffle -------------------------------------------------------------------------------- /graffles/StreamingRelationStrategy-apply.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/StreamingRelationStrategy-apply.graffle -------------------------------------------------------------------------------- /graffles/SymmetricHashJoinStateManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-structured-streaming-internals/b606c066a069ade151b56a28c3f0dcdc1f9a9266/graffles/SymmetricHashJoinStateManager.graffle -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | git+https://${GH_TOKEN}@github.com/squidfunk/mkdocs-material-insiders.git 2 | mkdocs-minify-plugin>=0.3.0 3 | mkdocs-git-revision-date-localized-plugin>=0.8 4 | mkdocs-git-revision-date-plugin>=0.3.1 5 | mkdocs-awesome-pages-plugin>=2.5.0 6 | mkdocs-redirects>=1.0.1 7 | mkdocs-macros-plugin>=0.5.0 8 | --------------------------------------------------------------------------------