├── .github ├── FUNDING.yml └── workflows │ └── deploying-to-github-pages.yml ├── .gitignore ├── LICENSE ├── README.md ├── docs ├── AggregatingAccumulator.md ├── Analyzer.md ├── BaseRelation.md ├── BaseSessionStateBuilder.md ├── BindReferences.md ├── CacheManager.md ├── Catalog.md ├── CatalogImpl.md ├── CatalogStatistics.md ├── CatalogStorageFormat.md ├── CatalogTable.md ├── CatalogTablePartition.md ├── CatalogUtils.md ├── CatalystSerde.md ├── CatalystTypeConverters.md ├── CheckAnalysis.md ├── Column.md ├── ColumnarRule.md ├── CommandUtils.md ├── CreatableRelationProvider.md ├── CreateTableWriter.md ├── DataFrame.md ├── DataFrameNaFunctions.md ├── DataFrameReader.md ├── DataFrameStatFunctions.md ├── DataFrameWriter.md ├── DataFrameWriterV2.md ├── DataSource.md ├── DataSourceRDD.md ├── DataSourceRDDPartition.md ├── DataSourceRegister.md ├── Dataset.md ├── DistinctKeyVisitor.md ├── Encoder.md ├── Encoders.md ├── ExecutionListenerBus.md ├── ExecutionListenerManager.md ├── ExperimentalMethods.md ├── ExplainUtils.md ├── ExpressionEncoder.md ├── ExternalAppendOnlyUnsafeRowArray.md ├── ExternalCatalog.md ├── ExternalCatalogWithListener.md ├── ExtractEquiJoinKeys.md ├── ExtractJoinWithBuckets.md ├── ExtractSingleColumnNullAwareAntiJoin.md ├── FileRelation.md ├── Filter.md ├── FilterEvaluatorFactory.md ├── FunctionRegistry.md ├── FunctionRegistryBase.md ├── GlobalTempViewManager.md ├── HashMapGenerator.md ├── InMemoryCatalog.md ├── InsertableRelation.md ├── InternalRow.md ├── IntervalUtils.md ├── JoinSelectionHelper.md ├── KeyValueGroupedDataset.md ├── KnownSizeEstimation.md ├── Observation.md ├── PhysicalOperation.md ├── PredicateHelper.md ├── PrunedFilteredScan.md ├── PrunedScan.md ├── PushDownUtils.md ├── QueryExecution.md ├── QueryExecutionListener.md ├── QueryPlanningTracker.md ├── RelationProvider.md ├── RelationalGroupedDataset.md ├── Row.md ├── RowEncoder.md ├── RuntimeConfig.md ├── SQLConf.md ├── SQLConfHelper.md ├── SQLExecution.md ├── SQLExecutionRDD.md ├── SQLMetric.md ├── ScalaReflection.md ├── SchemaRelationProvider.md ├── SerializerBuildHelper.md ├── SessionCatalog.md ├── SessionState.md ├── SessionStateBuilder.md ├── SharedState.md ├── ShuffledRowRDD.md ├── SimpleFunctionRegistry.md ├── SimpleFunctionRegistryBase.md ├── SimpleTableFunctionRegistry.md ├── SparkOptimizer.md ├── SparkPlanner.md ├── SparkSession-Builder.md ├── SparkSession.md ├── SparkSessionExtensions.md ├── StaticSQLConf.md ├── SubExprUtils.md ├── TableFunctionRegistry.md ├── TableScan.md ├── TypedColumn.md ├── UnsafeExternalRowSorter.md ├── UnsafeHashedRelation.md ├── UnsafeRow.md ├── V2SessionCatalog.md ├── WriteConfigMethods.md ├── adaptive-query-execution │ ├── .pages │ ├── AQEOptimizer.md │ ├── AQEUtils.md │ ├── AdaptiveExecutionContext.md │ ├── AdaptiveRulesHolder.md │ ├── CostEvaluator.md │ ├── ShufflePartitionsUtil.md │ ├── SimpleCostEvaluator.md │ └── index.md ├── aggregations │ ├── .pages │ ├── AggUtils.md │ ├── AggregationIterator.md │ ├── KVSorterIterator.md │ ├── ObjectAggregationIterator.md │ ├── ObjectAggregationMap.md │ ├── PhysicalAggregation.md │ ├── SortBasedAggregationIterator.md │ ├── TungstenAggregationIterator.md │ ├── UnsafeFixedWidthAggregationMap.md │ ├── UnsafeKVExternalSorter.md │ └── index.md ├── avro │ ├── AvroDataToCatalyst.md │ ├── AvroFileFormat.md │ ├── AvroOptions.md │ ├── CatalystDataToAvro.md │ └── index.md ├── bloom-filter-join │ ├── .pages │ ├── BloomFilter.md │ ├── BloomFilterImpl.md │ └── index.md ├── bucketing │ ├── .pages │ ├── BucketSpec.md │ └── index.md ├── cache-serialization │ ├── .pages │ ├── CachedBatch.md │ ├── CachedBatchSerializer.md │ ├── CachedRDDBuilder.md │ ├── DefaultCachedBatchSerializer.md │ ├── SimpleMetricsCachedBatch.md │ ├── SimpleMetricsCachedBatchSerializer.md │ └── index.md ├── caching-and-persistence.md ├── caching-webui-storage.md ├── catalyst-dsl │ ├── DslLogicalPlan.md │ └── index.md ├── catalyst │ ├── .pages │ ├── GenericStrategy.md │ ├── Optimizer.md │ ├── PlanChangeLogger.md │ ├── QueryPlan.md │ ├── QueryPlanner.md │ ├── Rule.md │ ├── RuleExecutor.md │ ├── TreeNode.md │ ├── TreePattern.md │ ├── TreePatternBits.md │ └── index.md ├── checkpointing.md ├── columnar-execution │ └── index.md ├── common-table-expressions │ ├── .pages │ └── index.md ├── configuration-properties.md ├── connector │ ├── .pages │ ├── ApplyTransform.md │ ├── Batch.md │ ├── BatchWrite.md │ ├── CustomMetric.md │ ├── DataSourceV2Implicits.md │ ├── DataWriter.md │ ├── DataWriterFactory.md │ ├── Expression.md │ ├── InputPartition.md │ ├── KeyGroupedPartitioning.md │ ├── LocalScan.md │ ├── OptionsHelper.md │ ├── PartitionReader.md │ ├── PartitionReaderFactory.md │ ├── PartitionSpecsHelper.md │ ├── Partitioning.md │ ├── Predicate.md │ ├── RewritableTransform.md │ ├── RowLevelOperation.md │ ├── Scan.md │ ├── ScanBuilder.md │ ├── SessionConfigSupport.md │ ├── SimpleTableProvider.md │ ├── StagedTable.md │ ├── SupportsAtomicPartitionManagement.md │ ├── SupportsDelete.md │ ├── SupportsDeleteV2.md │ ├── SupportsDelta.md │ ├── SupportsDynamicOverwrite.md │ ├── SupportsMetadata.md │ ├── SupportsMetadataColumns.md │ ├── SupportsOverwrite.md │ ├── SupportsPartitionManagement.md │ ├── SupportsPushDownAggregates.md │ ├── SupportsPushDownFilters.md │ ├── SupportsPushDownRequiredColumns.md │ ├── SupportsPushDownV2Filters.md │ ├── SupportsRead.md │ ├── SupportsReportOrdering.md │ ├── SupportsReportPartitioning.md │ ├── SupportsReportStatistics.md │ ├── SupportsRowLevelOperations.md │ ├── SupportsRuntimeFiltering.md │ ├── SupportsRuntimeV2Filtering.md │ ├── SupportsStreamingUpdate.md │ ├── SupportsTruncate.md │ ├── SupportsWrite.md │ ├── Table.md │ ├── TableCapability.md │ ├── TableHelper.md │ ├── TableProvider.md │ ├── Transform.md │ ├── TransformHelper.md │ ├── TruncatableTable.md │ ├── V1Scan.md │ ├── V1Table.md │ ├── V1WriteBuilder.md │ ├── Write.md │ ├── WriteBuilder.md │ ├── catalog │ │ ├── CatalogExtension.md │ │ ├── CatalogHelper.md │ │ ├── CatalogManager.md │ │ ├── CatalogPlugin.md │ │ ├── CatalogV2Util.md │ │ ├── Catalogs.md │ │ ├── Column.md │ │ ├── DelegatingCatalogExtension.md │ │ ├── FunctionCatalog.md │ │ ├── MetadataColumn.md │ │ ├── StagingTableCatalog.md │ │ ├── SupportsCatalogOptions.md │ │ ├── SupportsNamespaces.md │ │ ├── TableCatalog.md │ │ ├── TableCatalogCapability.md │ │ ├── TableChange.md │ │ ├── V2TableWithV1Fallback.md │ │ ├── ViewCatalog.md │ │ └── index.md │ ├── expressions │ │ ├── .pages │ │ ├── Aggregation.md │ │ ├── SortOrder.md │ │ └── index.md │ └── index.md ├── connectors │ ├── DDLUtils.md │ ├── DataSourceUtils.md │ ├── DataSourceV2Utils.md │ ├── DataWritingSparkTask.md │ ├── OutputWriter.md │ └── index.md ├── cost-based-optimization │ ├── BasicStatsPlanVisitor.md │ ├── CatalogColumnStat.md │ ├── ColumnStat.md │ ├── EstimationUtils.md │ ├── JoinEstimation.md │ ├── LogicalPlanStats.md │ ├── LogicalPlanVisitor.md │ ├── SizeInBytesOnlyStatsPlanVisitor.md │ ├── Statistics.md │ └── index.md ├── dataset │ ├── .pages │ ├── actions.md │ ├── basic-actions.md │ ├── index.md │ ├── typed-transformations.md │ └── untyped-transformations.md ├── debugging-query-execution.md ├── default-columns │ ├── .pages │ └── index.md ├── demo │ ├── .pages │ ├── adaptive-query-execution.md │ ├── connecting-spark-sql-to-hive-metastore.md │ ├── demo-multi-dimensional-aggregations.md │ ├── developing-catalogplugin.md │ ├── dynamic-partition-pruning.md │ ├── hive-partitioned-parquet-table-partition-pruning.md │ ├── index.md │ ├── objecthashaggregateexec-sort-based-fallback-tasks.md │ ├── spilling.md │ └── using-jdbc-data-source-to-access-postgresql.md ├── developer-api.md ├── direct-queries-on-files │ ├── .pages │ └── index.md ├── dynamic-partition-inserts.md ├── dynamic-partition-pruning │ └── index.md ├── execution-planning-strategies │ ├── .pages │ ├── Aggregation.md │ ├── BasicOperators.md │ ├── DataSourceStrategy.md │ ├── DataSourceV2Strategy.md │ ├── FileSourceStrategy.md │ ├── InMemoryScans.md │ ├── JoinSelection.md │ ├── LogicalQueryStageStrategy.md │ ├── SparkStrategies.md │ ├── SparkStrategy.md │ ├── SpecialLimits.md │ ├── Window.md │ ├── WithCTEStrategy.md │ └── index.md ├── expressions │ ├── AggregateExpression.md │ ├── AggregateFunction.md │ ├── AggregateWindowFunction.md │ ├── Aggregator.md │ ├── ArrayBasedSimpleHigherOrderFunction.md │ ├── ArrayFilter.md │ ├── Attribute.md │ ├── AttributeSeq.md │ ├── BasePredicate.md │ ├── BinaryComparison.md │ ├── BinaryOperator.md │ ├── BloomFilterAggregate.md │ ├── BloomFilterMightContain.md │ ├── BoundReference.md │ ├── CallMethodViaReflection.md │ ├── CodeGeneratorWithInterpretedFallback.md │ ├── CodegenFallback.md │ ├── Collect.md │ ├── CollectSet.md │ ├── Count.md │ ├── CreateNamedStruct.md │ ├── CreateStruct.md │ ├── CumeDist.md │ ├── DeclarativeAggregate.md │ ├── DecodeUsingSerializer.md │ ├── DynamicPruningExpression.md │ ├── DynamicPruningSubquery.md │ ├── EncodeUsingSerializer.md │ ├── EqualNullSafe.md │ ├── EqualTo.md │ ├── ExecSubqueryExpression-ScalarSubquery.md │ ├── ExecSubqueryExpression.md │ ├── Exists.md │ ├── ExpectsInputTypes.md │ ├── ExplodeBase.md │ ├── Expression.md │ ├── First.md │ ├── Generator.md │ ├── HashExpression.md │ ├── HashPartitioning.md │ ├── HigherOrderFunction.md │ ├── ImperativeAggregate.md │ ├── In.md │ ├── InSet.md │ ├── InSubquery.md │ ├── InSubqueryExec.md │ ├── Inline.md │ ├── InterpretedProjection.md │ ├── JsonToStructs.md │ ├── LessThanOrEqual.md │ ├── ListQuery.md │ ├── Literal.md │ ├── MapBasedSimpleHigherOrderFunction.md │ ├── MaxBy.md │ ├── MonotonicallyIncreasingID.md │ ├── Murmur3Hash.md │ ├── MutableProjection.md │ ├── NamedExpression.md │ ├── Nondeterministic.md │ ├── OffsetWindowFunction.md │ ├── ParseToDate.md │ ├── ParseToTimestamp.md │ ├── PlanExpression.md │ ├── Predicate.md │ ├── Projection.md │ ├── PythonUDF.md │ ├── RowNumber.md │ ├── RowNumberLike.md │ ├── RowOrdering.md │ ├── RuntimeReplaceable.md │ ├── ScalaAggregator.md │ ├── ScalaUDAF.md │ ├── ScalaUDF.md │ ├── ScalarSubquery.md │ ├── SimpleHigherOrderFunction.md │ ├── SimpleTypedAggregateExpression.md │ ├── SortOrder.md │ ├── SparkUserDefinedFunction.md │ ├── Stateful.md │ ├── StaticInvoke.md │ ├── SubqueryExpression.md │ ├── TimeWindow.md │ ├── TypedImperativeAggregate.md │ ├── UnaryExpression.md │ ├── Unevaluable.md │ ├── UnixTimestamp.md │ ├── UnresolvedAttribute.md │ ├── UnresolvedFunction.md │ ├── UnresolvedGenerator.md │ ├── UnresolvedOrdinal.md │ ├── UnresolvedStar.md │ ├── UnsafeProjection.md │ ├── UserDefinedAggregateFunction.md │ ├── UserDefinedAggregator.md │ ├── UserDefinedExpression.md │ ├── UserDefinedFunction.md │ ├── WindowExpression.md │ ├── WindowFunction.md │ ├── WindowSpecDefinition.md │ └── index.md ├── features │ └── index.md ├── file-based-data-scanning │ └── index.md ├── files │ ├── AggregatePushDownUtils.md │ ├── BaseDynamicPartitionDataWriter.md │ ├── BasicWriteJobStatsTracker.md │ ├── BasicWriteTaskStats.md │ ├── BasicWriteTaskStatsTracker.md │ ├── CatalogFileIndex.md │ ├── CompressionCodecs.md │ ├── DynamicPartitionDataConcurrentWriter.md │ ├── DynamicPartitionDataSingleWriter.md │ ├── FileBatchWrite.md │ ├── FileDataSourceV2.md │ ├── FileFormat.md │ ├── FileFormatDataWriter.md │ ├── FileFormatWriter.md │ ├── FileIndex.md │ ├── FilePartition.md │ ├── FilePartitionReaderFactory.md │ ├── FileScan.md │ ├── FileScanBuilder.md │ ├── FileStatusCache.md │ ├── FileTable.md │ ├── FileWrite.md │ ├── FileWriterFactory.md │ ├── HadoopFileLinesReader.md │ ├── HadoopFsRelation.md │ ├── InMemoryFileIndex.md │ ├── PartitionedFile.md │ ├── PartitionedFileUtil.md │ ├── PartitioningAwareFileIndex.md │ ├── PrunedInMemoryFileIndex.md │ ├── RecordReaderIterator.md │ ├── SchemaMergeUtils.md │ ├── SharedInMemoryCache.md │ ├── SingleDirectoryDataWriter.md │ ├── WriteJobStatsTracker.md │ ├── WriteTaskStats.md │ ├── WriteTaskStatsTracker.md │ └── index.md ├── generated-columns │ ├── .pages │ ├── GeneratedColumn.md │ └── index.md ├── hidden-file-metadata │ ├── .pages │ ├── MetadataAttribute.md │ └── index.md ├── hints │ ├── .pages │ ├── HintErrorHandler.md │ ├── HintInfo.md │ ├── JoinHint.md │ ├── JoinStrategyHint.md │ ├── index.md │ └── join-strategy-hints.md ├── hive-integration.md ├── hive │ ├── CreateHiveTableAsSelectCommand.md │ ├── DataSinks.md │ ├── DetermineTableStats.md │ ├── HadoopTableReader.md │ ├── HiveAnalysis.md │ ├── HiveClient.md │ ├── HiveClientImpl.md │ ├── HiveExternalCatalog.md │ ├── HiveFileFormat.md │ ├── HiveMetastoreCatalog.md │ ├── HiveSessionCatalog.md │ ├── HiveSessionStateBuilder.md │ ├── HiveTableRelation.md │ ├── HiveTableScanExec.md │ ├── HiveTableScans.md │ ├── HiveUtils.md │ ├── InsertIntoHiveDirCommand.md │ ├── InsertIntoHiveTable.md │ ├── IsolatedClientLoader.md │ ├── RelationConversions.md │ ├── ResolveHiveSerdeTable.md │ ├── SaveAsHiveFile.md │ ├── TableReader.md │ ├── configuration-properties.md │ ├── index.md │ └── spark-sql-hive-metastore.md ├── images │ ├── AQEOptimizer.png │ ├── DataWritingCommand-metrics.png │ ├── Dataset.png │ ├── HashAggregateExec-webui-details-for-query.png │ ├── KafkaOffsetReader-fetchSpecificOffsets.png │ ├── ObjectHashAggregateExec-no-sort-fallback-tasks.png │ ├── ObjectHashAggregateExec-sort-fallback-tasks-after-repartition.png │ ├── ObjectHashAggregateExec-sort-fallback-tasks.png │ ├── ObjectHashAggregateExec-webui-details-for-query.png │ ├── QueryExecution-execution-pipeline.png │ ├── ReorderJoin-createOrderedJoin-four-plans-after.png │ ├── ReorderJoin-createOrderedJoin-four-plans-before.png │ ├── ReorderJoin-createOrderedJoin-four-plans.png │ ├── ShuffleExchangeExec-webui.png │ ├── SortAggregateExec-webui-details-for-query.png │ ├── SparkPlan-execute.png │ ├── SparkPlan-executeTake.png │ ├── demo-spill-webui-details-for-stage-tasks.png │ ├── demo-spill-webui-details-for-stage.png │ ├── execute-pipeline.png │ ├── sort-spill.png │ ├── spark-sql-Analyzer.png │ ├── spark-sql-BroadcastExchangeExec-webui-details-for-query.png │ ├── spark-sql-BroadcastHashJoinExec-webui-query-details.png │ ├── spark-sql-BroadcastNestedLoopJoinExec-webui-details-for-query.png │ ├── spark-sql-CatalogImpl-createExternalTable.png │ ├── spark-sql-CatalogImpl.png │ ├── spark-sql-DataFrameWrite-insertInto-webui-query-details.png │ ├── spark-sql-FileSourceScanExec-webui-query-details.png │ ├── spark-sql-FilterExec-webui-details-for-query.png │ ├── spark-sql-GenerateExec-doExecute.png │ ├── spark-sql-GenerateExec-webui-details-for-query.png │ ├── spark-sql-GlobalTempViewManager-creating-instance.png │ ├── spark-sql-GlobalTempViewManager.png │ ├── spark-sql-HiveExternalCatalog.png │ ├── spark-sql-HiveMetastoreCatalog.png │ ├── spark-sql-HiveSessionCatalog.png │ ├── spark-sql-HiveSessionStateBuilder-SessionState.png │ ├── spark-sql-HiveSessionStateBuilder.png │ ├── spark-sql-InMemoryTableScanExec-webui-query-details.png │ ├── spark-sql-InputAdapter-doProduce.png │ ├── spark-sql-JDBCRelation-webui-query-details.png │ ├── spark-sql-KafkaWriter-write-webui.png │ ├── spark-sql-LocalTableScanExec-webui-query-details.png │ ├── spark-sql-RuntimeConfig.png │ ├── spark-sql-ScalaUDAF-initialize.png │ ├── spark-sql-ScalaUDAF-merge.png │ ├── spark-sql-ScalaUDAF-update.png │ ├── spark-sql-SessionCatalog.png │ ├── spark-sql-SessionState.png │ ├── spark-sql-ShuffledHashJoinExec-webui-query-details.png │ ├── spark-sql-SortMergeJoinExec-webui-query-details.png │ ├── spark-sql-SparkOptimizer.png │ ├── spark-sql-SparkSqlAstBuilder.png │ ├── spark-sql-SubqueryExec-webui-details-for-query.png │ ├── spark-sql-UserDefinedAggregateFunction.png │ ├── spark-sql-WholeStageCodegenExec-webui.png │ ├── spark-sql-WindowExec-webui-query-details.png │ ├── spark-sql-bucketing-sortmergejoin-bucketed-tables-no-exchanges.png │ ├── spark-sql-bucketing-sortmergejoin-filescans.png │ ├── spark-sql-bucketing-sortmergejoin-one-bucketed-table.png │ ├── spark-sql-bucketing-sortmergejoin-sorted-dataset-and-bucketed-table.png │ ├── spark-sql-caching-webui-storage.png │ ├── spark-sql-performance-tuning-groupBy-aggregation-case1.png │ ├── spark-sql-performance-tuning-groupBy-aggregation-case4.png │ ├── spark-sql-pivot-webui-scanning-jobs.png │ ├── spark-sql-pivot-webui.png │ ├── spark-sql-tungsten-webui-storage.png │ ├── spark-webui-storage.png │ └── webui-physical-plan.png ├── implicits.md ├── index.md ├── jdbc │ ├── .pages │ ├── AggregatedDialect.md │ ├── JDBCOptions.md │ ├── JDBCRDD.md │ ├── JDBCRelation.md │ ├── JDBCScan.md │ ├── JDBCScanBuilder.md │ ├── JDBCTableCatalog.md │ ├── JdbcDialect.md │ ├── JdbcDialects.md │ ├── JdbcRelationProvider.md │ ├── JdbcUtils.md │ └── index.md ├── joins.md ├── kafka │ ├── ConsumerStrategy.md │ ├── InternalKafkaConsumer.md │ ├── InternalKafkaProducerPool.md │ ├── JsonUtils.md │ ├── KafkaBatch.md │ ├── KafkaBatchWrite.md │ ├── KafkaBatchWriterFactory.md │ ├── KafkaDataConsumer.md │ ├── KafkaDataWriter.md │ ├── KafkaOffsetRangeLimit.md │ ├── KafkaOffsetReader.md │ ├── KafkaRecordToRowConverter.md │ ├── KafkaRelation.md │ ├── KafkaRowWriter.md │ ├── KafkaScan.md │ ├── KafkaSourceProvider.md │ ├── KafkaSourceRDD.md │ ├── KafkaSourceRDDPartition.md │ ├── KafkaTable.md │ ├── KafkaWrite.md │ ├── KafkaWriteTask.md │ ├── KafkaWriter.md │ ├── configuration-properties.md │ ├── index.md │ └── options.md ├── logical-analysis-rules │ ├── .pages │ ├── AddMetadataColumns.md │ ├── AliasViewChild.md │ ├── BindParameters.md │ ├── CTESubstitution.md │ ├── CleanupAliases.md │ ├── DataSourceAnalysis.md │ ├── ExtractWindowExpressions.md │ ├── FindDataSourceTable.md │ ├── LookupFunctions.md │ ├── PreWriteCheck.md │ ├── PreprocessTableCreation.md │ ├── ResolveAggregateFunctions.md │ ├── ResolveAliases.md │ ├── ResolveCatalogs.md │ ├── ResolveCoalesceHints.md │ ├── ResolveCreateNamedStruct.md │ ├── ResolveDefaultColumns.md │ ├── ResolveFunctions.md │ ├── ResolveGroupingAnalytics.md │ ├── ResolveInlineTables.md │ ├── ResolveInsertInto.md │ ├── ResolveJoinStrategyHints.md │ ├── ResolveOrdinalInOrderByAndGroupBy.md │ ├── ResolveReferences.md │ ├── ResolveRelations.md │ ├── ResolveSQLOnFile.md │ ├── ResolveSessionCatalog.md │ ├── ResolveSubquery.md │ ├── ResolveWindowFrame.md │ ├── ResolveWithCTE.md │ ├── RewriteDeleteFromTable.md │ ├── RewriteRowLevelCommand.md │ ├── TableCapabilityCheck.md │ ├── WindowFrameCoercion.md │ ├── WindowsSubstitution.md │ └── index.md ├── logical-operators │ ├── .pages │ ├── AddColumns.md │ ├── Aggregate.md │ ├── AlterTable.md │ ├── AlterTableAddColumnsCommand.md │ ├── AlterTableCommand.md │ ├── AnalysisOnlyCommand.md │ ├── AnalyzeColumn.md │ ├── AnalyzeColumnCommand.md │ ├── AnalyzePartitionCommand.md │ ├── AnalyzeTable.md │ ├── AnalyzeTableCommand.md │ ├── AnalyzeTablesCommand.md │ ├── AppendData.md │ ├── ArrowEvalPython.md │ ├── BaseEvalPython.md │ ├── CTERelationDef.md │ ├── CTERelationRef.md │ ├── CacheTableCommand.md │ ├── ClearCacheCommand.md │ ├── CollectMetrics.md │ ├── Command.md │ ├── CommentOnTable.md │ ├── CreateDataSourceTableAsSelectCommand.md │ ├── CreateDataSourceTableCommand.md │ ├── CreateNamespace.md │ ├── CreateTable.md │ ├── CreateTableAsSelect.md │ ├── CreateTempViewUsing.md │ ├── CreateView.md │ ├── CreateViewCommand.md │ ├── DataSourceV2Relation.md │ ├── DataSourceV2ScanRelation.md │ ├── DataWritingCommand.md │ ├── DeleteFromTable.md │ ├── DescribeColumnCommand.md │ ├── DescribeRelation.md │ ├── DescribeTableCommand.md │ ├── DeserializeToObject.md │ ├── DropNamespace.md │ ├── Except.md │ ├── Expand.md │ ├── ExplainCommand.md │ ├── ExposesMetadataColumns.md │ ├── ExternalRDD.md │ ├── FlatMapGroupsWithState.md │ ├── Generate.md │ ├── GlobalLimit.md │ ├── GroupingSets.md │ ├── IgnoreCachedData.md │ ├── InMemoryRelation.md │ ├── InsertIntoDataSourceCommand.md │ ├── InsertIntoDir.md │ ├── InsertIntoHadoopFsRelationCommand.md │ ├── InsertIntoStatement.md │ ├── InsertIntoTable.md │ ├── Intersect.md │ ├── Join.md │ ├── LeafNode.md │ ├── LeafRunnableCommand.md │ ├── LoadDataCommand.md │ ├── LocalRelation.md │ ├── LogicalPlan.md │ ├── LogicalPlanDistinctKeys.md │ ├── LogicalQueryStage.md │ ├── LogicalRDD.md │ ├── LogicalRelation.md │ ├── MapPartitions.md │ ├── MergeIntoTable.md │ ├── MultiInstanceRelation.md │ ├── NameParameterizedQuery.md │ ├── NamedRelation.md │ ├── Offset.md │ ├── OrderPreservingUnaryNode.md │ ├── OverwriteByExpression.md │ ├── OverwritePartitionsDynamic.md │ ├── ParameterizedQuery.md │ ├── ParsedStatement.md │ ├── Pivot.md │ ├── Project.md │ ├── RebalancePartitions.md │ ├── RelationTimeTravel.md │ ├── Repartition.md │ ├── RepartitionByExpression.md │ ├── RepartitionOperation.md │ ├── ReplaceData.md │ ├── ResolvedHint.md │ ├── ResolvedTable.md │ ├── RowLevelWrite.md │ ├── RunnableCommand.md │ ├── SaveIntoDataSourceCommand.md │ ├── SetCatalogAndNamespace.md │ ├── ShowColumns.md │ ├── ShowColumnsCommand.md │ ├── ShowCreateTable.md │ ├── ShowCreateTableCommand.md │ ├── ShowTableProperties.md │ ├── ShowTablePropertiesCommand.md │ ├── ShowTables.md │ ├── Sort.md │ ├── SubqueryAlias.md │ ├── SupportsSubquery.md │ ├── TruncateTableCommand.md │ ├── Union.md │ ├── UnresolvedCatalogRelation.md │ ├── UnresolvedHaving.md │ ├── UnresolvedHint.md │ ├── UnresolvedRelation.md │ ├── UnresolvedTable.md │ ├── UnresolvedTableOrView.md │ ├── UnresolvedTableValuedFunction.md │ ├── UnresolvedWith.md │ ├── UpdateTable.md │ ├── V1WriteCommand.md │ ├── V2CreateTablePlan.md │ ├── V2WriteCommand.md │ ├── View.md │ ├── Window.md │ ├── WithCTE.md │ ├── WithWindowDefinition.md │ ├── WriteDelta.md │ ├── WriteFiles.md │ └── index.md ├── logical-optimizations │ ├── .pages │ ├── AQEPropagateEmptyRelation.md │ ├── CleanupDynamicPruningFilters.md │ ├── CollapseWindow.md │ ├── ColumnPruning.md │ ├── CombineTypedFilters.md │ ├── CombineUnions.md │ ├── ComputeCurrentTime.md │ ├── ConstantFolding.md │ ├── ConvertToLocalRelation.md │ ├── CostBasedJoinReorder.md │ ├── DecimalAggregates.md │ ├── DynamicJoinSelection.md │ ├── EliminateResolvedHint.md │ ├── EliminateSerialization.md │ ├── EliminateSubqueryAliases.md │ ├── EliminateView.md │ ├── ExtractPythonUDFFromAggregate.md │ ├── ExtractPythonUDFs.md │ ├── GetCurrentDatabase.md │ ├── GroupBasedRowLevelOperationScanPlanning.md │ ├── InferFiltersFromConstraints.md │ ├── InjectRuntimeFilter.md │ ├── InlineCTE.md │ ├── LimitPushDown.md │ ├── NullPropagation.md │ ├── OptimizeIn.md │ ├── OptimizeMetadataOnlyQuery.md │ ├── OptimizeSubqueries.md │ ├── PartitionPruning.md │ ├── PropagateEmptyRelation.md │ ├── PruneFileSourcePartitions.md │ ├── PruneFilters.md │ ├── PruneHiveTablePartitions.md │ ├── PullupCorrelatedPredicates.md │ ├── PushDownLeftSemiAntiJoin.md │ ├── PushDownOperatorsToDataSource.md │ ├── PushDownPredicate.md │ ├── PushDownPredicates.md │ ├── PushPredicateThroughJoin.md │ ├── ReorderJoin.md │ ├── ReplaceExceptWithAntiJoin.md │ ├── ReplaceExceptWithFilter.md │ ├── ReplaceExpressions.md │ ├── RewriteCorrelatedScalarSubquery.md │ ├── RewriteExceptAll.md │ ├── RewritePredicateSubquery.md │ ├── SchemaPruning.md │ ├── SimplifyCasts.md │ ├── UpdateAttributeNullability.md │ ├── UpdateCTERelationStats.md │ ├── V2ScanRelationPushDown.md │ ├── V2Writes.md │ └── index.md ├── metadata-columns │ ├── .pages │ ├── FileSourceConstantMetadataAttribute.md │ ├── FileSourceGeneratedMetadataAttribute.md │ ├── FileSourceMetadataAttribute.md │ ├── MetadataColumnHelper.md │ ├── MetadataColumnsHelper.md │ └── index.md ├── named-function-arguments.md ├── new-and-noteworthy │ ├── catalog-plugin-api-and-multi-catalog-support.md │ ├── explain-command-improved.md │ ├── intervals.md │ ├── observable-metrics.md │ └── statistics.md ├── noop │ ├── NoopBatchWrite.md │ ├── NoopDataSource.md │ ├── NoopStreamingWrite.md │ ├── NoopTable.md │ ├── NoopWriteBuilder.md │ └── index.md ├── overview.md ├── parameterized-queries │ ├── .pages │ └── index.md ├── parquet │ ├── .pages │ ├── ParquetDataSourceV2.md │ ├── ParquetFileFormat.md │ ├── ParquetFilters.md │ ├── ParquetOptions.md │ ├── ParquetPartitionReaderFactory.md │ ├── ParquetReadSupport.md │ ├── ParquetScan.md │ ├── ParquetScanBuilder.md │ ├── ParquetTable.md │ ├── ParquetUtils.md │ ├── ParquetWrite.md │ ├── ParquetWriteSupport.md │ ├── SparkToParquetSchemaConverter.md │ ├── SpecificParquetRecordReaderBase.md │ ├── VectorizedColumnReader.md │ ├── VectorizedParquetRecordReader.md │ └── index.md ├── partition-file-metadata-caching │ └── index.md ├── physical-operators │ ├── AQEShuffleReadExec.md │ ├── AdaptiveSparkPlanExec.md │ ├── AggregateCodegenSupport.md │ ├── AliasAwareOutputExpression.md │ ├── AliasAwareQueryOutputOrdering.md │ ├── AllTuples.md │ ├── AlterTableExec.md │ ├── AtomicTableWriteExec.md │ ├── BaseAggregateExec.md │ ├── BaseCacheTableExec.md │ ├── BaseJoinExec.md │ ├── BaseSubqueryExec.md │ ├── BatchScanExec.md │ ├── BatchWriteHelper.md │ ├── BroadcastDistribution.md │ ├── BroadcastExchangeExec.md │ ├── BroadcastExchangeLike.md │ ├── BroadcastHashJoinExec.md │ ├── BroadcastMode.md │ ├── BroadcastNestedLoopJoinExec.md │ ├── BroadcastQueryStageExec.md │ ├── CacheTableAsSelectExec.md │ ├── CacheTableExec.md │ ├── ClusteredDistribution.md │ ├── CoalesceExec.md │ ├── CodegenSupport.md │ ├── CollectLimitExec.md │ ├── CollectMetricsExec.md │ ├── ColumnarToRowExec.md │ ├── ColumnarToRowTransition.md │ ├── CreateNamespaceExec.md │ ├── CreateTableAsSelectExec.md │ ├── DataSourceScanExec.md │ ├── DataSourceV2ScanExecBase.md │ ├── DataWritingCommandExec.md │ ├── DebugExec.md │ ├── DeleteFromTableExec.md │ ├── DescribeTableExec.md │ ├── DeserializeToObjectExec.md │ ├── Distribution.md │ ├── DropNamespaceExec.md │ ├── EvalPythonExec.md │ ├── Exchange.md │ ├── ExecutedCommandExec.md │ ├── ExpandExec.md │ ├── ExternalRDDScanExec.md │ ├── FileSourceScanExec.md │ ├── FilterExec.md │ ├── GenerateExec.md │ ├── HashAggregateExec.md │ ├── HashClusteredDistribution.md │ ├── HashJoin.md │ ├── HashedRelation.md │ ├── HashedRelationBroadcastMode.md │ ├── InMemoryTableScanExec.md │ ├── InputAdapter.md │ ├── JoinCodegenSupport.md │ ├── LocalTableScanExec.md │ ├── LongHashedRelation.md │ ├── ObjectConsumerExec.md │ ├── ObjectHashAggregateExec.md │ ├── ObjectProducerExec.md │ ├── OrderPreservingUnaryExecNode.md │ ├── OrderedDistribution.md │ ├── OverwriteByExpressionExec.md │ ├── Partitioning.md │ ├── PartitioningPreservingUnaryExecNode.md │ ├── ProjectExec.md │ ├── QueryStageExec.md │ ├── RangeExec.md │ ├── ReusedExchangeExec.md │ ├── ReusedSubqueryExec.md │ ├── RowDataSourceScanExec.md │ ├── RowToColumnarExec.md │ ├── SerializeFromObjectExec.md │ ├── SetCatalogAndNamespaceExec.md │ ├── ShowCreateTableExec.md │ ├── ShowTablePropertiesExec.md │ ├── ShowTablesExec.md │ ├── ShuffleExchangeExec.md │ ├── ShuffleExchangeLike.md │ ├── ShuffleOrigin.md │ ├── ShuffleQueryStageExec.md │ ├── ShuffleSpec.md │ ├── ShuffledHashJoinExec.md │ ├── ShuffledJoin.md │ ├── SortAggregateExec.md │ ├── SortExec.md │ ├── SortMergeJoinExec.md │ ├── SortMergeJoinScanner.md │ ├── SparkPlan.md │ ├── SubqueryExec.md │ ├── TableWriteExecHelper.md │ ├── TruncateTableExec.md │ ├── UnaryExecNode.md │ ├── UnspecifiedDistribution.md │ ├── V2CommandExec.md │ ├── V2ExistingTableWriteExec.md │ ├── V2TableWriteExec.md │ ├── WholeStageCodegenExec.md │ ├── WindowExec.md │ ├── WindowExecBase.md │ ├── WriteDeltaExec.md │ ├── WriteFilesExec.md │ └── index.md ├── physical-optimizations │ ├── .pages │ ├── AQEShuffleReadRule.md │ ├── AdjustShuffleExchangePosition.md │ ├── ApplyColumnarRulesAndInsertTransitions.md │ ├── CoalesceBucketsInJoin.md │ ├── CoalesceShufflePartitions.md │ ├── CollapseCodegenStages.md │ ├── DisableUnnecessaryBucketedScan.md │ ├── EnsureRequirements.md │ ├── InsertAdaptiveSparkPlan.md │ ├── OptimizeShuffleWithLocalRead.md │ ├── OptimizeSkewInRebalancePartitions.md │ ├── OptimizeSkewedJoin.md │ ├── PlanAdaptiveDynamicPruningFilters.md │ ├── PlanAdaptiveSubqueries.md │ ├── PlanDynamicPruningFilters.md │ ├── PlanSubqueries.md │ ├── RemoveRedundantProjects.md │ ├── RemoveRedundantSorts.md │ ├── ReplaceHashWithSortAgg.md │ ├── ReuseAdaptiveSubquery.md │ ├── ReuseExchange.md │ ├── ReuseExchangeAndSubquery.md │ ├── ReuseSubquery.md │ ├── ValidateSparkPlan.md │ └── index.md ├── query-execution │ └── index.md ├── rdds │ └── FileScanRDD.md ├── runtime-filtering │ ├── .pages │ └── index.md ├── spark-connect.md ├── spark-logging.md ├── spark-sql-column-operators.md ├── spark-sql-dataset-rdd.md ├── spark-sql-dataset-vs-sql.md ├── spark-sql-joins-broadcast.md ├── spark-sql-performance-tuning-groupBy-aggregation.md ├── spark-sql-performance-tuning.md ├── spark-sql-udfs-blackbox.md ├── spark-sql-udfs.md ├── sql │ ├── AbstractSqlParser.md │ ├── AstBuilder.md │ ├── CatalystSqlParser.md │ ├── ParserInterface.md │ ├── SparkSqlAstBuilder.md │ ├── SparkSqlParser.md │ ├── VariableSubstitution.md │ └── index.md ├── standard-functions │ ├── .pages │ ├── aggregate.md │ ├── collection-functions.md │ ├── datetime.md │ ├── index.md │ ├── regular-functions.md │ └── windows-functions.md ├── storage-partitioned-joins │ ├── .pages │ └── index.md ├── subexpression-elimination │ ├── .pages │ ├── EquivalentExpressions.md │ └── index.md ├── subqueries │ ├── .pages │ └── index.md ├── table-valued-functions │ ├── .pages │ └── index.md ├── thrift-server │ ├── .pages │ ├── SparkSQLEnv.md │ └── index.md ├── time-travel │ ├── .pages │ ├── TimeTravelSpec.md │ └── index.md ├── transactional-writes │ ├── .pages │ ├── SQLHadoopMapReduceCommitProtocol.md │ └── index.md ├── tungsten │ ├── UnsafeRowSerializerInstance.md │ └── index.md ├── types │ ├── .pages │ ├── AbstractDataType.md │ ├── ArrayType.md │ ├── AtomicType.md │ ├── CalendarInterval.md │ ├── DataType.md │ ├── Metadata.md │ ├── MetadataBuilder.md │ ├── StructField.md │ ├── StructType.md │ ├── UserDefinedType.md │ └── index.md ├── ui │ ├── .pages │ ├── AllExecutionsPage.md │ ├── ExecutionPage.md │ ├── SQLAppStatusListener.md │ ├── SQLAppStatusStore.md │ ├── SQLTab.md │ ├── SparkListenerSQLExecutionEnd.md │ └── index.md ├── user-defined-functions │ ├── .pages │ ├── UDFRegistration.md │ ├── UserDefinedPythonFunction.md │ └── index.md ├── variable-substitution.md ├── vectorized-decoding │ ├── ColumnVector.md │ ├── OffHeapColumnVector.md │ ├── OnHeapColumnVector.md │ ├── WritableColumnVector.md │ └── index.md ├── vectorized-query-execution │ ├── ColumnarBatch.md │ └── index.md ├── whole-stage-code-generation │ ├── .pages │ ├── Block.md │ ├── BufferedRowIterator.md │ ├── CodeGenerator.md │ ├── CodegenContext.md │ ├── GenerateColumnAccessor.md │ ├── GenerateMutableProjection.md │ ├── GenerateOrdering.md │ ├── GeneratePredicate.md │ ├── GenerateSafeProjection.md │ ├── GenerateUnsafeProjection.md │ └── index.md └── window-functions │ ├── .pages │ ├── AggregateProcessor.md │ ├── RangeFrame.md │ ├── Window.md │ ├── WindowFunctionFrame.md │ ├── WindowSpec.md │ └── index.md ├── graffles ├── AQEOptimizer.graffle ├── Dataset.graffle ├── QueryExecution-execution-pipeline.graffle ├── ReorderJoin-createOrderedJoin-four-plans-after.graffle ├── ReorderJoin-createOrderedJoin-four-plans-before.graffle ├── ReorderJoin-createOrderedJoin-four-plans.graffle ├── spark-sql-Analyzer.graffle ├── spark-sql-CatalogImpl-createExternalTable.graffle ├── spark-sql-CatalogImpl.graffle ├── spark-sql-GenerateExec-doExecute.graffle ├── spark-sql-GlobalTempViewManager-creating-instance.graffle ├── spark-sql-GlobalTempViewManager.graffle ├── spark-sql-HiveExternalCatalog.graffle ├── spark-sql-HiveMetastoreCatalog.graffle ├── spark-sql-HiveSessionCatalog.graffle ├── spark-sql-HiveSessionStateBuilder-SessionState.graffle ├── spark-sql-HiveSessionStateBuilder.graffle ├── spark-sql-InputAdapter-doProduce.graffle ├── spark-sql-RuntimeConfig.graffle ├── spark-sql-ScalaUDAF-initialize.graffle ├── spark-sql-ScalaUDAF-merge.graffle ├── spark-sql-ScalaUDAF-update.graffle ├── spark-sql-SessionCatalog.graffle ├── spark-sql-SessionState.graffle ├── spark-sql-SparkOptimizer.graffle ├── spark-sql-SparkPlan-execute-pipeline.graffle ├── spark-sql-SparkPlan-execute.graffle ├── spark-sql-SparkPlan-executeTake.graffle ├── spark-sql-SparkSqlAstBuilder.graffle └── spark-sql-UserDefinedAggregateFunction.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 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # The Internals of Spark SQL Online Book 2 | 3 | [![Deploying to GitHub Pages](https://github.com/japila-books/spark-sql-internals/actions/workflows/deploying-to-github-pages.yml/badge.svg)](https://github.com/japila-books/spark-sql-internals/actions/workflows/deploying-to-github-pages.yml) 4 | 5 | The project contains the sources of [The Internals of Spark SQL](https://books.japila.pl/spark-sql-internals/) online book. 6 | -------------------------------------------------------------------------------- /docs/Catalog.md: -------------------------------------------------------------------------------- 1 | # Catalog — Metastore Management Interface 2 | 3 | `Catalog` is an [abstraction](#contract) of [metadata catalogs](#implementations) for managing relational entities (e.g. database(s), tables, functions, table columns and temporary views). 4 | 5 | `Catalog` is available using [SparkSession.catalog](SparkSession.md#catalog) property. 6 | 7 | ```scala 8 | assert(spark.isInstanceOf[org.apache.spark.sql.SparkSession]) 9 | assert(spark.catalog.isInstanceOf[org.apache.spark.sql.catalog.Catalog]) 10 | ``` 11 | 12 | ## Contract 13 | 14 | ### cacheTable 15 | 16 | ```scala 17 | cacheTable( 18 | tableName: String): Unit 19 | cacheTable( 20 | tableName: String, 21 | storageLevel: StorageLevel): Unit 22 | ``` 23 | 24 | Caches a given table 25 | 26 | Used for SQL's [CACHE TABLE](caching-and-persistence.md#cache-table) and `AlterTableRenameCommand` command. 27 | 28 | ### Others 29 | 30 | ## Implementations 31 | 32 | * [CatalogImpl](CatalogImpl.md) 33 | -------------------------------------------------------------------------------- /docs/CatalogUtils.md: -------------------------------------------------------------------------------- 1 | # CatalogUtils 2 | 3 | ## Normalizing BucketSpec 4 | 5 | ```scala 6 | normalizeBucketSpec( 7 | tableName: String, 8 | tableCols: Seq[String], 9 | bucketSpec: BucketSpec, 10 | resolver: Resolver): BucketSpec 11 | ``` 12 | 13 | `normalizeBucketSpec`...FIXME 14 | 15 | --- 16 | 17 | `normalizeBucketSpec` is used when: 18 | 19 | * [PreprocessTableCreation](logical-analysis-rules/PreprocessTableCreation.md) logical analysis rule is executed (on a bucketed table while appending data to a [CreateTable](logical-operators/CreateTable.md)) 20 | -------------------------------------------------------------------------------- /docs/CatalystTypeConverters.md: -------------------------------------------------------------------------------- 1 | # CatalystTypeConverters Helper Object 2 | 3 | `CatalystTypeConverters` is a Scala object that is used to convert Scala types to Catalyst types and vice versa. 4 | 5 | ## Creating Catalyst Converter 6 | 7 | ```scala 8 | createToCatalystConverter( 9 | dataType: DataType): Any => Any 10 | ``` 11 | 12 | `createToCatalystConverter`...FIXME 13 | 14 | `createToCatalystConverter` is used when: 15 | 16 | * FIXME 17 | 18 | === [[convertToCatalyst]] `convertToCatalyst` Method 19 | 20 | [source, scala] 21 | ---- 22 | convertToCatalyst(a: Any): Any 23 | ---- 24 | 25 | `convertToCatalyst`...FIXME 26 | 27 | NOTE: `convertToCatalyst` is used when...FIXME 28 | -------------------------------------------------------------------------------- /docs/CreatableRelationProvider.md: -------------------------------------------------------------------------------- 1 | # CreatableRelationProvider 2 | 3 | `CreatableRelationProvider` is an [abstraction](#contract) of [data source providers](#implementations) that can [save data and create a BaseRelation](#createRelation). 4 | 5 | ## Contract 6 | 7 | ### createRelation 8 | 9 | ```scala 10 | createRelation( 11 | sqlContext: SQLContext, 12 | mode: SaveMode, 13 | parameters: Map[String, String], 14 | data: DataFrame): BaseRelation 15 | ``` 16 | 17 | Saves the given `DataFrame` to this data source (and creates a [BaseRelation](BaseRelation.md) to represent the relation) 18 | 19 | The `SaveMode` specifies what should happen when the target relation (destination) already exists. 20 | 21 | Used when: 22 | 23 | * `DataSource` is requested to [writeAndRead](DataSource.md#writeAndRead) 24 | * `SaveIntoDataSourceCommand` logical command is requested to [run](logical-operators/SaveIntoDataSourceCommand.md#run) 25 | 26 | ## Implementations 27 | 28 | * `ConsoleSinkProvider` 29 | * [JdbcRelationProvider](jdbc/JdbcRelationProvider.md) 30 | * [KafkaSourceProvider](kafka/KafkaSourceProvider.md) 31 | -------------------------------------------------------------------------------- /docs/DataFrameNaFunctions.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: DataFrameNaFunctions 3 | subtitle: Working With Missing Data 4 | --- 5 | 6 | # DataFrameNaFunctions — Working With Missing Data 7 | 8 | `DataFrameNaFunctions` is used to work with missing data in a [DataFrame](DataFrame.md). 9 | 10 | `DataFrameNaFunctions` is available using [na](dataset/untyped-transformations.md#na) untyped transformation. 11 | 12 | ```scala 13 | val q: DataFrame = ... 14 | q.na 15 | ``` 16 | -------------------------------------------------------------------------------- /docs/DataSourceRDDPartition.md: -------------------------------------------------------------------------------- 1 | # DataSourceRDDPartition 2 | 3 | `DataSourceRDDPartition` is a `Partition` ([Apache Spark]({{ book.spark_core }}/rdd/Partition)) of [DataSourceRDD](DataSourceRDD.md). 4 | 5 | ## Creating Instance 6 | 7 | `DataSourceRDDPartition` takes the following to be created: 8 | 9 | * Partition Index 10 | * [InputPartition](connector/InputPartition.md)s 11 | 12 | `DataSourceRDDPartition` is created when: 13 | 14 | * `DataSourceRDD` is requested for the [partitions](DataSourceRDD.md#getPartitions) 15 | -------------------------------------------------------------------------------- /docs/DataSourceRegister.md: -------------------------------------------------------------------------------- 1 | # DataSourceRegister 2 | 3 | `DataSourceRegister` is an [abstraction](#contract) of data sources to be available under [shortName](#shortName) alias (so it can be [looked up](DataSource.md#lookupDataSource) by the alias not a fully-qualified class name) 4 | 5 | ## Contract 6 | 7 | ###  Short Name 8 | 9 | ```scala 10 | shortName(): String 11 | ``` 12 | 13 | Short name (_alias_) of the data source 14 | 15 | Used when: 16 | 17 | * `DataSource` utility is used to [lookup a data source](DataSource.md#lookupDataSource) 18 | 19 | ## Data Source Discovery 20 | 21 | `DataSourceRegister` should register itself in `META-INF/services/org.apache.spark.sql.sources.DataSourceRegister` file for Java's [ServiceLoader]({{ java.api }}/java/util/ServiceLoader.html#load(java.lang.Class,java.lang.ClassLoader)) to discover the service. 22 | -------------------------------------------------------------------------------- /docs/ExternalCatalogWithListener.md: -------------------------------------------------------------------------------- 1 | # ExternalCatalogWithListener 2 | 3 | `ExternalCatalogWithListener` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/FileRelation.md: -------------------------------------------------------------------------------- 1 | # FileRelation 2 | 3 | `FileRelation` is an [abstraction](#contract) of [relations](#implementations) that are backed by files. 4 | 5 | ## Contract 6 | 7 | ### inputFiles 8 | 9 | ```scala 10 | inputFiles: Array[String] 11 | ``` 12 | 13 | The files that this relation will read when scanning 14 | 15 | Used when: 16 | 17 | * `Dataset` is requested for the [inputFiles](dataset/index.md#inputFiles) 18 | 19 | ## Implementations 20 | 21 | * [HadoopFsRelation](files/HadoopFsRelation.md) 22 | -------------------------------------------------------------------------------- /docs/FilterEvaluatorFactory.md: -------------------------------------------------------------------------------- 1 | # FilterEvaluatorFactory 2 | 3 | `FilterEvaluatorFactory` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/InMemoryCatalog.md: -------------------------------------------------------------------------------- 1 | # InMemoryCatalog 2 | 3 | `InMemoryCatalog` is...FIXME 4 | 5 | === [[listPartitionsByFilter]] `listPartitionsByFilter` Method 6 | 7 | [source, scala] 8 | ---- 9 | listPartitionsByFilter( 10 | db: String, 11 | table: String, 12 | predicates: Seq[Expression], 13 | defaultTimeZoneId: String): Seq[CatalogTablePartition] 14 | ---- 15 | 16 | `listPartitionsByFilter`...FIXME 17 | 18 | `listPartitionsByFilter` is part of the [ExternalCatalog](ExternalCatalog.md#listPartitionsByFilter) abstraction. 19 | -------------------------------------------------------------------------------- /docs/InsertableRelation.md: -------------------------------------------------------------------------------- 1 | # InsertableRelation 2 | 3 | `InsertableRelation` is an [abstraction](#contract) of [relations](#implementations) with support for [inserting or overwriting data](#insert). 4 | 5 | ## Contract 6 | 7 | ###  Inserting Data into or Overwriting Relation 8 | 9 | ```scala 10 | insert( 11 | data: DataFrame, 12 | overwrite: Boolean): Unit 13 | ``` 14 | 15 | Inserts or overwrites data (from the given [DataFrame](DataFrame.md)) 16 | 17 | Used when: 18 | 19 | * [InsertIntoDataSourceCommand](logical-operators/InsertIntoDataSourceCommand.md) logical command is executed 20 | * `SupportsV1Write` physical operator is executed 21 | 22 | ## Implementations 23 | 24 | * [JDBCRelation](jdbc/JDBCRelation.md) 25 | -------------------------------------------------------------------------------- /docs/IntervalUtils.md: -------------------------------------------------------------------------------- 1 | # IntervalUtils 2 | 3 | ## Parsing CalendarInterval 4 | 5 | ```scala 6 | fromIntervalString( 7 | input: String): CalendarInterval 8 | ``` 9 | 10 | `fromIntervalString`...FIXME 11 | 12 | `fromIntervalString` is used when: 13 | 14 | * `TimeWindow` utility is used to [getIntervalInMicroSeconds](expressions/TimeWindow.md#getIntervalInMicroSeconds) 15 | * `Dataset` is requested to [withWatermark](dataset/index.md#withWatermark) 16 | -------------------------------------------------------------------------------- /docs/KnownSizeEstimation.md: -------------------------------------------------------------------------------- 1 | # KnownSizeEstimation 2 | 3 | `KnownSizeEstimation` is an [abstraction](#contract) of [size estimators](#implementations) that can give a more precise [size estimation](#estimatedSize). 4 | 5 | ## Contract 6 | 7 | ### Estimated Size 8 | 9 | ```scala 10 | estimatedSize: Long 11 | ``` 12 | 13 | `estimatedSize` is used when: 14 | 15 | * `SizeEstimator` is requested to `visitSingleObject` 16 | * [BroadcastExchangeExec](physical-operators/BroadcastExchangeExec.md) physical operator is requested for [relationFuture](physical-operators/BroadcastExchangeExec.md#relationFuture) 17 | * [BroadcastHashJoinExec](physical-operators/BroadcastHashJoinExec.md) physical operator is [executed](physical-operators/BroadcastHashJoinExec.md#doExecute) 18 | * [ShuffledHashJoinExec](physical-operators/ShuffledHashJoinExec.md) physical operator is requested to [buildHashedRelation](physical-operators/ShuffledHashJoinExec.md#buildHashedRelation) 19 | 20 | ## Implementations 21 | 22 | * [HashedRelation](physical-operators/HashedRelation.md) 23 | -------------------------------------------------------------------------------- /docs/PrunedScan.md: -------------------------------------------------------------------------------- 1 | # PrunedScan 2 | 3 | `PrunedScan` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/QueryExecutionListener.md: -------------------------------------------------------------------------------- 1 | # QueryExecutionListener 2 | 3 | `QueryExecutionListener` is an [abstraction](#contract) of [query execution listeners](#implementations) that can intercept [onFailure](#onFailure) and [onSuccess](#onSuccess) events. 4 | 5 | ## Contract 6 | 7 | ### onFailure 8 | 9 | ```scala 10 | onFailure( 11 | funcName: String, 12 | qe: QueryExecution, 13 | exception: Exception): Unit 14 | ``` 15 | 16 | Used when: 17 | 18 | * `ExecutionListenerBus` is requested to [doPostEvent](ExecutionListenerBus.md#doPostEvent) 19 | 20 | ### onSuccess 21 | 22 | ```scala 23 | onSuccess( 24 | funcName: String, 25 | qe: QueryExecution, 26 | durationNs: Long): Unit 27 | ``` 28 | 29 | Used when: 30 | 31 | * `ExecutionListenerBus` is requested to [doPostEvent](ExecutionListenerBus.md#doPostEvent) 32 | 33 | ## Implementations 34 | 35 | * `ObservationListener` 36 | -------------------------------------------------------------------------------- /docs/SQLConfHelper.md: -------------------------------------------------------------------------------- 1 | # SQLConfHelper 2 | 3 | `SQLConfHelper` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/SessionStateBuilder.md: -------------------------------------------------------------------------------- 1 | # SessionStateBuilder 2 | 3 | `SessionStateBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/SimpleFunctionRegistry.md: -------------------------------------------------------------------------------- 1 | # SimpleFunctionRegistry 2 | 3 | `SimpleFunctionRegistry` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/SimpleTableFunctionRegistry.md: -------------------------------------------------------------------------------- 1 | # SimpleTableFunctionRegistry 2 | 3 | `SimpleTableFunctionRegistry` is a [TableFunctionRegistry](TableFunctionRegistry.md) and [SimpleFunctionRegistryBase](SimpleFunctionRegistryBase.md) (of [LogicalPlan](logical-operators/LogicalPlan.md)s). 4 | 5 | ## Creating Instance 6 | 7 | `SimpleTableFunctionRegistry` takes no arguments to be created. 8 | 9 | `SimpleTableFunctionRegistry` is created when: 10 | 11 | * `TableFunctionRegistry` is requested for the [builtin TableFunctionRegistry](TableFunctionRegistry.md#builtin) 12 | -------------------------------------------------------------------------------- /docs/UnsafeHashedRelation.md: -------------------------------------------------------------------------------- 1 | # UnsafeHashedRelation 2 | 3 | `UnsafeHashedRelation` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/WriteConfigMethods.md: -------------------------------------------------------------------------------- 1 | # WriteConfigMethods 2 | 3 | `WriteConfigMethods` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/adaptive-query-execution/.pages: -------------------------------------------------------------------------------- 1 | title: Adaptive Query Execution 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/adaptive-query-execution/CostEvaluator.md: -------------------------------------------------------------------------------- 1 | # CostEvaluator 2 | 3 | `CostEvaluator` is an [abstraction](#contract) of [cost evaluators](#implementations) in [Adaptive Query Execution](index.md). 4 | 5 | `CostEvaluator` is used in [AdaptiveSparkPlanExec](../physical-operators/AdaptiveSparkPlanExec.md#costEvaluator) physical operator based on [spark.sql.adaptive.customCostEvaluatorClass](../configuration-properties.md#spark.sql.adaptive.customCostEvaluatorClass) configuration property. 6 | 7 | ## Contract 8 | 9 | ###  Evaluating Cost 10 | 11 | ```scala 12 | evaluateCost( 13 | plan: SparkPlan): Cost 14 | ``` 15 | 16 | Evaluates the cost of the given [SparkPlan](../physical-operators/SparkPlan.md) 17 | 18 | Used when: 19 | 20 | * `AdaptiveSparkPlanExec` physical operator is requested for the [final physical query plan](../physical-operators/AdaptiveSparkPlanExec.md#getFinalPhysicalPlan) 21 | 22 | ## Implementations 23 | 24 | * [SimpleCostEvaluator](SimpleCostEvaluator.md) 25 | -------------------------------------------------------------------------------- /docs/adaptive-query-execution/SimpleCostEvaluator.md: -------------------------------------------------------------------------------- 1 | # SimpleCostEvaluator 2 | 3 | `SimpleCostEvaluator` is a [CostEvaluator](CostEvaluator.md) in [Adaptive Query Execution](index.md). 4 | 5 | ## Evaluating Cost 6 | 7 | ```scala 8 | evaluateCost( 9 | plan: SparkPlan): Cost 10 | ``` 11 | 12 | `evaluateCost` is part of the [CostEvaluator](CostEvaluator.md#evaluateCost) abstraction. 13 | 14 | --- 15 | 16 | `evaluateCost` counts the [shuffle exchanges](../physical-operators/ShuffleExchangeLike.md) unary physical operators in the given [SparkPlan](../physical-operators/SparkPlan.md). 17 | -------------------------------------------------------------------------------- /docs/aggregations/.pages: -------------------------------------------------------------------------------- 1 | title: Aggregate Queries 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/aggregations/KVSorterIterator.md: -------------------------------------------------------------------------------- 1 | # KVSorterIterator 2 | 3 | `KVSorterIterator` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/aggregations/ObjectAggregationMap.md: -------------------------------------------------------------------------------- 1 | # ObjectAggregationMap 2 | 3 | `ObjectAggregationMap` is an in-memory map to store aggregation buffer for hash-based aggregation (using [ObjectAggregationIterator](ObjectAggregationIterator.md)). 4 | -------------------------------------------------------------------------------- /docs/aggregations/UnsafeKVExternalSorter.md: -------------------------------------------------------------------------------- 1 | # UnsafeKVExternalSorter 2 | 3 | `UnsafeKVExternalSorter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/bloom-filter-join/.pages: -------------------------------------------------------------------------------- 1 | title: Bloom Filter Join 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/bucketing/.pages: -------------------------------------------------------------------------------- 1 | title: Bucketing 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/cache-serialization/.pages: -------------------------------------------------------------------------------- 1 | title: Cache Serialization 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/cache-serialization/CachedBatch.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # CachedBatch 7 | 8 | `CachedBatch` is an [abstraction](#contract) of [cached batches of data](#implementations) with the [numRows](#numRows) and [sizeInBytes](#sizeInBytes) metrics. 9 | 10 | ## Contract 11 | 12 | ### numRows 13 | 14 | ```scala 15 | numRows: Int 16 | ``` 17 | 18 | Used when: 19 | 20 | * `CachedRDDBuilder` is requested to [buildBuffers](CachedRDDBuilder.md#buildBuffers) 21 | * `InMemoryTableScanExec` physical operator is requested for the [inputRDD](../physical-operators/InMemoryTableScanExec.md#inputRDD) 22 | 23 | ### sizeInBytes 24 | 25 | ```scala 26 | sizeInBytes: Long 27 | ``` 28 | 29 | Used when: 30 | 31 | * `CachedRDDBuilder` is requested to [buildBuffers](CachedRDDBuilder.md#buildBuffers) 32 | 33 | ## Implementations 34 | 35 | * [SimpleMetricsCachedBatch](SimpleMetricsCachedBatch.md) 36 | -------------------------------------------------------------------------------- /docs/cache-serialization/SimpleMetricsCachedBatch.md: -------------------------------------------------------------------------------- 1 | # SimpleMetricsCachedBatch 2 | 3 | `SimpleMetricsCachedBatch` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/cache-serialization/SimpleMetricsCachedBatchSerializer.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # SimpleMetricsCachedBatchSerializer 7 | 8 | `SimpleMetricsCachedBatchSerializer` is a base abstraction of the [CachedBatchSerializer](CachedBatchSerializer.md) abstraction for [serializers](#implementations) with the default [buildFilter](#buildFilter). 9 | 10 | ## Implementations 11 | 12 | * [DefaultCachedBatchSerializer](DefaultCachedBatchSerializer.md) 13 | 14 | ## Building Batch Filter 15 | 16 | ??? note "Signature" 17 | 18 | ```scala 19 | buildFilter( 20 | predicates: Seq[Expression], 21 | cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] 22 | ``` 23 | 24 | `buildFilter` is part of the [CachedBatchSerializer](CachedBatchSerializer.md#buildFilter) abstraction. 25 | 26 | `buildFilter`...FIXME 27 | -------------------------------------------------------------------------------- /docs/cache-serialization/index.md: -------------------------------------------------------------------------------- 1 | # Cache Serialization 2 | 3 | [SPARK-32274](https://issues.apache.org/jira/browse/SPARK-32274) introduced pluggable **Cache Serialization** based upon the following abstractions: 4 | 5 | * [CachedBatch](CachedBatch.md) 6 | * [CachedBatchSerializer](CachedBatchSerializer.md) 7 | -------------------------------------------------------------------------------- /docs/catalyst/.pages: -------------------------------------------------------------------------------- 1 | title: Catalyst 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/catalyst/index.md: -------------------------------------------------------------------------------- 1 | # Catalyst Tree Manipulation Framework 2 | 3 | **Catalyst** is an execution-agnostic framework to represent and manipulate a **dataflow graph** as [trees](TreeNode.md) of [relational operators](QueryPlan.md) and [expressions](../expressions/Expression.md). 4 | 5 | The Catalyst framework was introduced in [[SPARK-1251] Support for optimizing and executing structured queries](https://issues.apache.org/jira/browse/SPARK-1251). 6 | 7 | Spark SQL uses the Catalyst framework to build an extensible [Optimizer](Optimizer.md) with a number of built-in [logical query plan optimizations](Optimizer.md#defaultBatches). 8 | 9 | Catalyst supports both rule-based and cost-based optimizations. 10 | -------------------------------------------------------------------------------- /docs/common-table-expressions/.pages: -------------------------------------------------------------------------------- 1 | title: Common Table Expressions 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/common-table-expressions/index.md: -------------------------------------------------------------------------------- 1 | # Common Table Expressions 2 | 3 | **Common Table Expressions** (_CTEs_) are defined using the [WITH](../sql/AstBuilder.md#withCTE) clause: 4 | 5 | ```text 6 | WITH namedQuery (',' namedQuery)* 7 | 8 | namedQuery 9 | : name (columnAliases)? AS? '(' query ')' 10 | ; 11 | ``` 12 | 13 | CTEs allow for _statement scoped views_ that a user can reference (possibly multiple times) within the scope of a SQL statement. 14 | 15 | ## References 16 | 17 | * [Common Table Expression (CTE)](http://spark.apache.org/docs/latest/sql-ref-syntax-qry-select-cte.html) by the offical Spark documentation 18 | * [Common table expression](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL#Common_table_expression) by Wikipedia 19 | * [with — Organize Complex Queries](https://modern-sql.com/feature/with) 20 | -------------------------------------------------------------------------------- /docs/connector/.pages: -------------------------------------------------------------------------------- 1 | title: Connector API 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/connector/ApplyTransform.md: -------------------------------------------------------------------------------- 1 | # ApplyTransform 2 | 3 | `ApplyTransform` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/CustomMetric.md: -------------------------------------------------------------------------------- 1 | # CustomMetric 2 | 3 | `CustomMetric` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/DataSourceV2Implicits.md: -------------------------------------------------------------------------------- 1 | # DataSourceV2Implicits 2 | 3 | `DataSourceV2Implicits` is a Scala object with the following Scala implicit classes: 4 | 5 | * [MetadataColumnHelper](../metadata-columns/MetadataColumnHelper.md) 6 | * [MetadataColumnsHelper](../metadata-columns/MetadataColumnsHelper.md) 7 | * [OptionsHelper](OptionsHelper.md) 8 | * [PartitionSpecsHelper](PartitionSpecsHelper.md) 9 | * [TableHelper](TableHelper.md) 10 | 11 | `DataSourceV2Implicits` is part of the `org.apache.spark.sql.execution.datasources.v2` package. 12 | 13 | ```scala 14 | import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits 15 | ``` 16 | -------------------------------------------------------------------------------- /docs/connector/DataWriterFactory.md: -------------------------------------------------------------------------------- 1 | # DataWriterFactory 2 | 3 | `DataWriterFactory` is an [abstraction](#contract) of [DataWriter factories](#implementations). 4 | 5 | `DataWriterFactory` is `Serializable`. 6 | 7 | ## Contract 8 | 9 | ### Creating DataWriter 10 | 11 | ```java 12 | DataWriter createWriter( 13 | int partitionId, 14 | long taskId) 15 | ``` 16 | 17 | Creates a [DataWriter](DataWriter.md) (for the given `partitionId` and `taskId`) 18 | 19 | Used when: 20 | 21 | * `DataWritingSparkTask` is requested to [run](../connectors/DataWritingSparkTask.md#run) 22 | 23 | ## Implementations 24 | 25 | * [FileWriterFactory](../files/FileWriterFactory.md) 26 | * [KafkaBatchWriterFactory](../kafka/KafkaBatchWriterFactory.md) 27 | * `MemoryWriterFactory` (Spark Structured Streaming) 28 | * `MicroBatchWriterFactory` (Spark Structured Streaming) 29 | * `NoopWriterFactory` 30 | -------------------------------------------------------------------------------- /docs/connector/Expression.md: -------------------------------------------------------------------------------- 1 | # Expression 2 | 3 | `Expression` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/KeyGroupedPartitioning.md: -------------------------------------------------------------------------------- 1 | # KeyGroupedPartitioning 2 | 3 | `KeyGroupedPartitioning` is a [Partitioning](Partitioning.md) where rows are split across partitions based on the [partition transform expressions](#keys). 4 | 5 | `KeyGroupedPartitioning` is a key part of [Storage-Partitioned Joins](../storage-partitioned-joins/index.md). 6 | 7 | !!! note 8 | Not used in any of the [built-in Spark SQL connectors](../connectors/index.md) yet. 9 | 10 | ## Creating Instance 11 | 12 | `KeyGroupedPartitioning` takes the following to be created: 13 | 14 | * Partition transform [expression](../expressions/Expression.md)s 15 | * Number of partitions 16 | -------------------------------------------------------------------------------- /docs/connector/LocalScan.md: -------------------------------------------------------------------------------- 1 | # LocalScan 2 | 3 | `LocalScan` is an [extension](#contract) of the [Scan](Scan.md) abstraction for [local scans](#implementations). 4 | 5 | `LocalScan` is planned as [LocalTableScanExec](../physical-operators/LocalTableScanExec.md) physical operator at execution planning. 6 | 7 | ## Contract 8 | 9 | ### rows 10 | 11 | ```java 12 | InternalRow[] rows() 13 | ``` 14 | 15 | Used when: 16 | 17 | * [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (on a [DataSourceV2ScanRelation](../logical-operators/DataSourceV2ScanRelation.md) logical operator with a `LocalScan`) 18 | 19 | ## Implementations 20 | 21 | !!! note 22 | No built-in implementations available. 23 | -------------------------------------------------------------------------------- /docs/connector/OptionsHelper.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: OptionsHelper 3 | --- 4 | 5 | # OptionsHelper Implicit Class 6 | 7 | `OptionsHelper` is a Scala implicit class for [Map[String, String]](#options). 8 | 9 | ## Creating Instance 10 | 11 | `OptionsHelper` takes the following to be created: 12 | 13 | * Options (`Map[String, String]`) 14 | 15 | ## asOptions 16 | 17 | ```scala 18 | asOptions: CaseInsensitiveStringMap 19 | ``` 20 | 21 | `asOptions` creates a new `CaseInsensitiveStringMap` with the [options](#options). 22 | 23 | `asOptions` is used when: 24 | 25 | * FIXME 26 | -------------------------------------------------------------------------------- /docs/connector/PartitionSpecsHelper.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: PartitionSpecsHelper 3 | --- 4 | 5 | # PartitionSpecsHelper Implicit Class 6 | 7 | `PartitionSpecsHelper` is a Scala implicit class for [Seq[PartitionSpec]](#partSpecs). 8 | 9 | ## Creating Instance 10 | 11 | `PartitionSpecsHelper` takes the following to be created: 12 | 13 | * `PartitionSpec`s 14 | -------------------------------------------------------------------------------- /docs/connector/Predicate.md: -------------------------------------------------------------------------------- 1 | # Predicate 2 | 3 | `Predicate` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/RewritableTransform.md: -------------------------------------------------------------------------------- 1 | # RewritableTransform 2 | 3 | `RewritableTransform` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/RowLevelOperation.md: -------------------------------------------------------------------------------- 1 | # RowLevelOperation 2 | 3 | `RowLevelOperation` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SessionConfigSupport.md: -------------------------------------------------------------------------------- 1 | # SessionConfigSupport 2 | 3 | `SessionConfigSupport` is an [extension](#contract) of the [TableProvider](TableProvider.md) abstraction for [table providers](#implementations) that use [custom key prefix for spark.datasource configuration options](#keyPrefix). 4 | 5 | `SessionConfigSupport` connectors can be configured by additional (session-scoped) configuration options that are specified in [SparkSession](../SparkSession.md) to extend user-defined options. 6 | 7 | ## Contract 8 | 9 | ### Configuration Key Prefix 10 | 11 | ```java 12 | String keyPrefix() 13 | ``` 14 | 15 | The prefix of the configuration keys of this connector that is added to `spark.datasource` prefix 16 | 17 | ```text 18 | spark.datasource.[keyPrefix] 19 | ``` 20 | 21 | Must not be `null` 22 | 23 | Used when: 24 | 25 | * `DataSourceV2Utils` is requested to [extract session configuration options](../connectors/DataSourceV2Utils.md#extractSessionConfigs) 26 | 27 | ## Implementations 28 | 29 | !!! note 30 | No built-in implementations available. 31 | -------------------------------------------------------------------------------- /docs/connector/StagedTable.md: -------------------------------------------------------------------------------- 1 | # StagedTable 2 | 3 | `StagedTable` is an [extension](#contract) of the [Table](Table.md) abstraction for tables that can [abort](#abortStagedChanges) or [commit](#commitStagedChanges) staged changes. 4 | 5 | ## Contract 6 | 7 | ###  abortStagedChanges 8 | 9 | ```java 10 | void abortStagedChanges() 11 | ``` 12 | 13 | Used when: 14 | 15 | * `AtomicReplaceTableExec` physical command is executed 16 | * `TableWriteExecHelper` is requested to [writeToTable](../physical-operators/TableWriteExecHelper.md#writeToTable) 17 | 18 | ###  commitStagedChanges 19 | 20 | ```java 21 | void commitStagedChanges() 22 | ``` 23 | 24 | Used when: 25 | 26 | * `AtomicReplaceTableExec` physical command is `executed` 27 | * `TableWriteExecHelper` is requested to [writeToTable](../physical-operators/TableWriteExecHelper.md#writeToTable) 28 | -------------------------------------------------------------------------------- /docs/connector/SupportsAtomicPartitionManagement.md: -------------------------------------------------------------------------------- 1 | # SupportsAtomicPartitionManagement 2 | 3 | `SupportsAtomicPartitionManagement` is an [extension](#contract) of the [SupportsPartitionManagement](SupportsPartitionManagement.md) abstraction for partitioned tables. 4 | 5 | ## Contract 6 | 7 | ###  createPartitions 8 | 9 | ```java 10 | void createPartitions( 11 | InternalRow[] idents, 12 | Map[] properties) 13 | ``` 14 | 15 | Used when: 16 | 17 | * `AlterTableAddPartitionExec` physical operator is executed 18 | 19 | ###  dropPartitions 20 | 21 | ```java 22 | boolean dropPartitions( 23 | InternalRow[] idents) 24 | ``` 25 | 26 | Used when: 27 | 28 | * `AlterTableDropPartitionExec` physical operator is executed 29 | -------------------------------------------------------------------------------- /docs/connector/SupportsDelete.md: -------------------------------------------------------------------------------- 1 | # SupportsDelete 2 | 3 | `SupportsDelete` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SupportsDelta.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SupportsDelta 3 | --- 4 | 5 | # SupportsDelta Row-Level Operations 6 | 7 | `SupportsDelta` is an [extension](#contract) of the [RowLevelOperation](RowLevelOperation.md) abstraction for [row-level operations](#implementations) with [rowId](#rowId). 8 | 9 | ## Contract (Subset) 10 | 11 | ### rowId { #rowId } 12 | 13 | ```java 14 | NamedReference[] rowId() 15 | ``` 16 | 17 | row ID column references for row equality 18 | 19 | Used when: 20 | 21 | * [RewriteRowLevelCommand](../logical-analysis-rules/RewriteRowLevelCommand.md) logical analysis rule is executed (and [resolveRowIdAttrs](../logical-analysis-rules/RewriteRowLevelCommand.md#resolveRowIdAttrs)) 22 | * [WriteDelta](../logical-operators/WriteDelta.md) logical operator is executed (and [rowIdAttrsResolved](../logical-operators/WriteDelta.md#rowIdAttrsResolved)) 23 | 24 | ## Implementations 25 | 26 | !!! note 27 | No built-in implementations available. 28 | -------------------------------------------------------------------------------- /docs/connector/SupportsDynamicOverwrite.md: -------------------------------------------------------------------------------- 1 | # SupportsDynamicOverwrite 2 | 3 | `SupportsDynamicOverwrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SupportsMetadataColumns.md: -------------------------------------------------------------------------------- 1 | # SupportsMetadataColumns 2 | 3 | `SupportsMetadataColumns` is an [extension](#contract) of the [Table](Table.md) abstraction for [tables](#implementations) with [metadata columns](#metadataColumns). 4 | 5 | ## Contract 6 | 7 | ###  metadataColumns 8 | 9 | ```java 10 | MetadataColumn[] metadataColumns() 11 | ``` 12 | 13 | [MetadataColumn](catalog/MetadataColumn.md)s of this table 14 | 15 | Used when: 16 | 17 | * `DataSourceV2Relation` is requested for the [metadata output](../logical-operators/DataSourceV2Relation.md#metadataOutput) 18 | * [DescribeTableExec](../physical-operators/DescribeTableExec.md) physical command is [executed](../physical-operators/DescribeTableExec.md#addMetadataColumns) 19 | 20 | ## Implementations 21 | 22 | !!! note 23 | No built-in implementations available. 24 | -------------------------------------------------------------------------------- /docs/connector/SupportsOverwrite.md: -------------------------------------------------------------------------------- 1 | # SupportsOverwrite 2 | 3 | `SupportsOverwrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SupportsPushDownRequiredColumns.md: -------------------------------------------------------------------------------- 1 | # SupportsPushDownRequiredColumns 2 | 3 | `SupportsPushDownRequiredColumns` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SupportsPushDownV2Filters.md: -------------------------------------------------------------------------------- 1 | # SupportsPushDownV2Filters 2 | 3 | `SupportsPushDownV2Filters` is an [extension](#contract) of the [ScanBuilder](ScanBuilder.md) abstraction for [scan builders](#implementations) that support [pushPredicates](#pushPredicates) (using "modern" [Predicate](Predicate.md)s). 4 | 5 | ## Contract 6 | 7 | ### pushedPredicates 8 | 9 | ```java 10 | Predicate[] pushedPredicates() 11 | ``` 12 | 13 | Used when: 14 | 15 | * `PushDownUtils` is requested to [pushFilters](../PushDownUtils.md#pushFilters) 16 | 17 | ### pushPredicates 18 | 19 | ```java 20 | Predicate[] pushPredicates( 21 | Predicate[] predicates) 22 | ``` 23 | 24 | Used when: 25 | 26 | * `PushDownUtils` is requested to [pushFilters](../PushDownUtils.md#pushFilters) 27 | 28 | ## Implementations 29 | 30 | * [JDBCScanBuilder](../jdbc/JDBCScanBuilder.md) 31 | -------------------------------------------------------------------------------- /docs/connector/SupportsReportOrdering.md: -------------------------------------------------------------------------------- 1 | # SupportsReportOrdering 2 | 3 | `SupportsReportOrdering` is an [extension](#contract) of the [Scan](Scan.md) abstraction for [scans](#implementations) that report the [order of data](#outputOrdering) (in each partition). 4 | 5 | ## Contract 6 | 7 | ### outputOrdering 8 | 9 | ```java 10 | SortOrder[] outputOrdering() 11 | ``` 12 | 13 | [SortOrder](expressions/SortOrder.md)s of the output ordering of this scan 14 | 15 | Used when: 16 | 17 | * `V2ScanPartitioningAndOrdering` logical optimization is executed (and `ordering`) 18 | 19 | ## Implementations 20 | 21 | !!! note 22 | No built-in implementations available. 23 | -------------------------------------------------------------------------------- /docs/connector/SupportsReportPartitioning.md: -------------------------------------------------------------------------------- 1 | # SupportsReportPartitioning 2 | 3 | `SupportsReportPartitioning` is an [extension](#contract) of the [Scan](Scan.md) abstraction for [scans](#implementations) with custom [data partitioning](#outputOrdering). 4 | 5 | ## Contract 6 | 7 | ### outputOrdering 8 | 9 | ```java 10 | Partitioning outputPartitioning() 11 | ``` 12 | 13 | [Partitioning](Partitioning.md)s of the data of this scan 14 | 15 | Used when: 16 | 17 | * `V2ScanPartitioningAndOrdering` logical optimization is executed (and `partitioning`) 18 | 19 | ## Implementations 20 | 21 | !!! note 22 | No built-in implementations available. 23 | -------------------------------------------------------------------------------- /docs/connector/SupportsReportStatistics.md: -------------------------------------------------------------------------------- 1 | # SupportsReportStatistics 2 | 3 | `SupportsReportStatistics` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/SupportsRowLevelOperations.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SupportsRowLevelOperations 3 | --- 4 | 5 | # SupportsRowLevelOperations Tables 6 | 7 | `SupportsRowLevelOperations` is an [extension](#contract) of the [Table](Table.md) abstraction for [tables](#implementations) that can [create a new RowLevelOperationBuilder](#newRowLevelOperationBuilder). 8 | 9 | ## Contract 10 | 11 | ### newRowLevelOperationBuilder 12 | 13 | ```java 14 | RowLevelOperationBuilder newRowLevelOperationBuilder( 15 | RowLevelOperationInfo info) 16 | ``` 17 | 18 | `RowLevelOperationBuilder` for the `RowLevelOperationInfo` 19 | 20 | Used when: 21 | 22 | * `RewriteRowLevelCommand` analysis rule is requested to [buildOperationTable](../logical-analysis-rules/RewriteRowLevelCommand.md#buildOperationTable) 23 | 24 | ## Implementations 25 | 26 | !!! note 27 | No built-in implementations available. 28 | -------------------------------------------------------------------------------- /docs/connector/SupportsRuntimeV2Filtering.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SupportsRuntimeV2Filtering 3 | --- 4 | 5 | # SupportsRuntimeV2Filtering 6 | 7 | `SupportsRuntimeV2Filtering` is an [extension](#contract) of the [Scan](Scan.md) abstraction for [connectors](#implementations) that can filter initially planned [InputPartition](InputPartition.md)s using predicates Spark infers at runtime. 8 | 9 | ## Contract 10 | 11 | ### filterAttributes { #filterAttributes } 12 | 13 | ```java 14 | NamedReference[] filterAttributes() 15 | ``` 16 | 17 | ### filter 18 | 19 | ```java 20 | void filter( 21 | Predicate[] predicates) 22 | ``` 23 | 24 | ## Implementations 25 | 26 | * [SupportsRuntimeFiltering](SupportsRuntimeFiltering.md) 27 | -------------------------------------------------------------------------------- /docs/connector/SupportsStreamingUpdate.md: -------------------------------------------------------------------------------- 1 | # SupportsStreamingUpdate 2 | 3 | `SupportsStreamingUpdate` is an [extension](#contract) of the [WriteBuilder](WriteBuilder.md) abstraction for [tables](#implementations) that support [streaming update](#update). 4 | 5 | ## Contract 6 | 7 | ###  Streaming Update 8 | 9 | ```scala 10 | update(): WriteBuilder 11 | ``` 12 | 13 | [WriteBuilder](WriteBuilder.md) 14 | 15 | Used when: 16 | 17 | * `StreamExecution` stream execution engine (Spark Structured Streaming) is requested to `createStreamingWrite` 18 | 19 | ## Implementations 20 | 21 | * ConsoleTable 22 | * ForeachWriterTable 23 | * [KafkaTable](../kafka/KafkaTable.md) 24 | * MemorySink 25 | * [NoopWriteBuilder](../noop/NoopWriteBuilder.md) 26 | -------------------------------------------------------------------------------- /docs/connector/SupportsTruncate.md: -------------------------------------------------------------------------------- 1 | # SupportsTruncate 2 | 3 | `SupportsTruncate` is an [extension](#contract) of the [WriteBuilder](WriteBuilder.md) abstraction for [tables](#implementations) that support [truncation](#truncate). 4 | 5 | ## Contract 6 | 7 | ###  Truncation 8 | 9 | ```java 10 | WriteBuilder truncate() 11 | ``` 12 | 13 | [WriteBuilder](WriteBuilder.md) that can replace all existing data with data committed in the write 14 | 15 | Used when: 16 | 17 | * [OverwriteByExpressionExec](../physical-operators/OverwriteByExpressionExec.md) and `OverwriteByExpressionExecV1` physical operators are executed 18 | * `StreamExecution` stream execution engine (Spark Structured Streaming) is requested to `createStreamingWrite` 19 | 20 | ## Implementations 21 | 22 | * ConsoleTable (Spark Structured Streaming) 23 | * ForeachWriterTable (Spark Structured Streaming) 24 | * [KafkaTable](../kafka/KafkaTable.md) 25 | * MemorySink (Spark Structured Streaming) 26 | * [NoopWriteBuilder](../noop/NoopWriteBuilder.md) 27 | * [SupportsOverwrite](SupportsOverwrite.md) 28 | -------------------------------------------------------------------------------- /docs/connector/Transform.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: Transform 3 | --- 4 | 5 | # Transform Function Expressions 6 | 7 | `Transform` is an [extension](#contract) of the [Expression](../expressions/Expression.md) abstraction for [transform functions](#implementations). 8 | 9 | ## Contract 10 | 11 | ###  Arguments 12 | 13 | ```java 14 | Expression[] arguments() 15 | ``` 16 | 17 | [Expression](../expressions/Expression.md)s of the arguments of this transform function 18 | 19 | ###  Name 20 | 21 | ```java 22 | String name() 23 | ``` 24 | 25 | ###  References 26 | 27 | ```java 28 | NamedReference[] references() 29 | ``` 30 | 31 | ## Implementations 32 | 33 | * [ApplyTransform](ApplyTransform.md) 34 | * [RewritableTransform](RewritableTransform.md) 35 | -------------------------------------------------------------------------------- /docs/connector/TransformHelper.md: -------------------------------------------------------------------------------- 1 | # TransformHelper 2 | 3 | `TransformHelper` is a Scala implicit class to extend [Seq[Transform]](#transforms) with [convertTransforms](#convertTransforms) extension method. 4 | 5 | ## Creating Instance 6 | 7 | `TransformHelper` takes the following to be created: 8 | 9 | * [Transform](Transform.md)s 10 | 11 | ## convertTransforms 12 | 13 | ```scala 14 | convertTransforms: (Seq[String], Option[BucketSpec]) 15 | ``` 16 | 17 | `convertTransforms`...FIXME 18 | 19 | --- 20 | 21 | `convertTransforms` is used when: 22 | 23 | * [ResolveSessionCatalog](../logical-analysis-rules/ResolveSessionCatalog.md) logical analysis rule is [executed](../logical-analysis-rules/ResolveSessionCatalog.md#buildCatalogTable) 24 | * `V2SessionCatalog` is requested to [createTable](../V2SessionCatalog.md#createTable) (_deprecated_) 25 | * `CatalogImpl` is requested to [listColumns](../CatalogImpl.md#listColumns) 26 | -------------------------------------------------------------------------------- /docs/connector/TruncatableTable.md: -------------------------------------------------------------------------------- 1 | # TruncatableTable 2 | 3 | `TruncatableTable` is an [extension](#contract) of the [Table](Table.md) abstraction for [tables](#implementations) that can be [truncated](#truncateTable) (i.e., with all rows removed from the table). 4 | 5 | ## Contract 6 | 7 | ### truncateTable { #truncateTable } 8 | 9 | ```java 10 | boolean truncateTable() 11 | ``` 12 | 13 | See: 14 | 15 | * [SupportsDeleteV2](SupportsDeleteV2.md#truncateTable) 16 | 17 | Used when: 18 | 19 | * [TruncateTableExec](../physical-operators/TruncateTableExec.md) physical operator is executed 20 | 21 | ## Implementations 22 | 23 | * [SupportsDeleteV2](SupportsDeleteV2.md) 24 | -------------------------------------------------------------------------------- /docs/connector/V1Scan.md: -------------------------------------------------------------------------------- 1 | # V1Scan 2 | 3 | `V1Scan` is an [extension](#contract) of the [Scan](Scan.md) abstraction for [V1 DataSources](#implementations) that would like to participate in the DataSource V2 read code paths. 4 | 5 | ## Contract 6 | 7 | ###  toV1TableScan 8 | 9 | ```java 10 | T toV1TableScan( 11 | SQLContext context) 12 | ``` 13 | 14 | [BaseRelation](../BaseRelation.md) with [TableScan](../TableScan.md) to scan data from a DataSource v1 (to `RDD[Row]`) 15 | 16 | Used when: 17 | 18 | * [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (to plan [DataSourceV2ScanRelation](../logical-operators/DataSourceV2ScanRelation.md) with `V1Scan` to [RowDataSourceScanExec](../physical-operators/RowDataSourceScanExec.md)) 19 | 20 | ## Implementations 21 | 22 | * [JDBCScan](../jdbc/JDBCScan.md) 23 | -------------------------------------------------------------------------------- /docs/connector/V1Table.md: -------------------------------------------------------------------------------- 1 | # V1Table 2 | 3 | `V1Table` is a [Table](Table.md) that acts as an adapter to expose [v1 table metadata](#v1Table) ([CatalogTable](../CatalogTable.md)). 4 | 5 | ## Creating Instance 6 | 7 | `V1Table` takes the following to be created: 8 | 9 | * [v1 Table Metadata](../CatalogTable.md) 10 | 11 | `V1Table` is created when: 12 | 13 | * `V2SessionCatalog` is requested to [load a table](../V2SessionCatalog.md#loadTable) 14 | 15 | ## String Representation { #toString } 16 | 17 | `toString` is the following string (with the [name](#name)): 18 | 19 | ```text 20 | V1Table([name]) 21 | ``` 22 | -------------------------------------------------------------------------------- /docs/connector/V1WriteBuilder.md: -------------------------------------------------------------------------------- 1 | # V1WriteBuilder 2 | 3 | `V1WriteBuilder` is an [extension](#contract) of the [WriteBuilder](WriteBuilder.md) abstraction for [V1 DataSources](#implementations) that would like to leverage the DataSource V2 write code paths. 4 | 5 | ## Contract 6 | 7 | ###  buildForV1Write 8 | 9 | ```java 10 | InsertableRelation buildForV1Write() 11 | ``` 12 | 13 | [InsertableRelation](../InsertableRelation.md) 14 | 15 | Used when: 16 | 17 | * `AppendDataExecV1`, `OverwriteByExpressionExecV1`, [CreateTableAsSelectExec](../physical-operators/CreateTableAsSelectExec.md), `ReplaceTableAsSelectExec` and [AtomicTableWriteExec](../physical-operators/AtomicTableWriteExec.md) physical commands are executed 18 | 19 | ## Implementations 20 | 21 | !!! note 22 | No known native Spark SQL implementations. 23 | -------------------------------------------------------------------------------- /docs/connector/catalog/CatalogExtension.md: -------------------------------------------------------------------------------- 1 | # CatalogExtension 2 | 3 | `CatalogExtension` is an [extension](#contract) of the [TableCatalog](TableCatalog.md) and [SupportsNamespaces](SupportsNamespaces.md) abstractions for [session catalog extensions](#implementations) that [setDelegateCatalog](#setDelegateCatalog). 4 | 5 | ## Contract 6 | 7 | ### setDelegateCatalog { #setDelegateCatalog } 8 | 9 | ```java 10 | void setDelegateCatalog( 11 | CatalogPlugin delegate) 12 | ``` 13 | 14 | Used when: 15 | 16 | * `CatalogManager` is requested to [loadV2SessionCatalog](CatalogManager.md#loadV2SessionCatalog) 17 | 18 | ## Implementations 19 | 20 | * [DelegatingCatalogExtension](DelegatingCatalogExtension.md) 21 | -------------------------------------------------------------------------------- /docs/connector/catalog/Column.md: -------------------------------------------------------------------------------- 1 | # Column 2 | 3 | `Column` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/catalog/SupportsCatalogOptions.md: -------------------------------------------------------------------------------- 1 | # SupportsCatalogOptions 2 | 3 | `SupportsCatalogOptions` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/catalog/TableCatalogCapability.md: -------------------------------------------------------------------------------- 1 | # TableCatalogCapability 2 | 3 | `TableCatalogCapability` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/catalog/V2TableWithV1Fallback.md: -------------------------------------------------------------------------------- 1 | # V2TableWithV1Fallback Tables 2 | 3 | `V2TableWithV1Fallback` is an [extension](#contract) of the [Table](../Table.md) abstraction for [tables](#implementations) with [V1 fallback support](#v1Table) (using [CatalogTable](../../CatalogTable.md)). 4 | 5 | ## Contract 6 | 7 | ### v1Table 8 | 9 | ```scala 10 | v1Table: CatalogTable 11 | ``` 12 | 13 | [CatalogTable](../../CatalogTable.md) to fall back to for unsupported V2 capabilities (that are supported in V1) 14 | 15 | Used when: 16 | 17 | * `ResolveRelations` logical resolution rule is requested to [createRelation](../../logical-analysis-rules/ResolveRelations.md#createRelation) (for a streaming table) 18 | * `DataStreamWriter` ([Spark Structured Streaming]({{ book.structured_streaming }}/DataStreamWriter)) is requested to `toTable` 19 | 20 | ## Implementations 21 | 22 | !!! note 23 | No known native Spark SQL implementations. 24 | -------------------------------------------------------------------------------- /docs/connector/catalog/ViewCatalog.md: -------------------------------------------------------------------------------- 1 | # ViewCatalog 2 | 3 | `ViewCatalog` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/catalog/index.md: -------------------------------------------------------------------------------- 1 | # Catalog Plugin API 2 | 3 | Main abstractions: 4 | 5 | * [CatalogManager](CatalogManager.md) 6 | * [CatalogPlugin](CatalogPlugin.md) 7 | * [FunctionCatalog](FunctionCatalog.md) 8 | 9 | ## spark.sql.catalog { #spark.sql.catalog } 10 | 11 | Custom [CatalogPlugin](CatalogPlugin.md)s are registered using `spark.sql.catalog` configuration property. 12 | 13 | ```text 14 | spark.sql.catalog.catalog-name=com.example.YourCatalogClass 15 | ``` 16 | 17 | Whenever additional features are required (e.g., [FunctionCatalog](FunctionCatalog.md)), the implicit class [CatalogHelper](CatalogHelper.md) is used to make the conversion. 18 | -------------------------------------------------------------------------------- /docs/connector/expressions/.pages: -------------------------------------------------------------------------------- 1 | title: Connector Expressions 2 | -------------------------------------------------------------------------------- /docs/connector/expressions/Aggregation.md: -------------------------------------------------------------------------------- 1 | # Aggregation Expression 2 | 3 | `Aggregation` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/expressions/SortOrder.md: -------------------------------------------------------------------------------- 1 | # SortOrder Expression 2 | 3 | `SortOrder` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/connector/expressions/index.md: -------------------------------------------------------------------------------- 1 | # Connector Expressions 2 | -------------------------------------------------------------------------------- /docs/connector/index.md: -------------------------------------------------------------------------------- 1 | # Connector API 2 | 3 | **Connector API** is a new API in Spark 3 for Spark SQL developers to create [connectors](../connectors/index.md) (_V2 data sources_ or _providers_). 4 | 5 | !!! note 6 | Connector API is meant to replace the older (deprecated) DataSource v1 and v2. 7 | 8 | Although "Data Source V2" name has already been used, Connector API is considered the "real" Data Source V2. 9 | 10 | [ResolveSessionCatalog](../logical-analysis-rules/ResolveSessionCatalog.md) logical resolution rule uses [TableProvider](TableProvider.md) to recognize Data Source V2 providers. 11 | 12 | [spark.sql.sources.useV1SourceList](../configuration-properties.md#spark.sql.sources.useV1SourceList) configuration property is used for connectors for which Data Source V2 code path is disabled (that should fall back to Data Source V1 execution paths). 13 | -------------------------------------------------------------------------------- /docs/connectors/DataSourceUtils.md: -------------------------------------------------------------------------------- 1 | # DataSourceUtils 2 | 3 | ## checkFieldNames { #checkFieldNames } 4 | 5 | ```scala 6 | checkFieldNames( 7 | format: FileFormat, 8 | schema: StructType): Unit 9 | ``` 10 | 11 | ??? warning "Procedure" 12 | `checkFieldNames` is a procedure (returns `Unit`) so _what happens inside stays inside_ (paraphrasing the [former advertising slogan of Las Vegas, Nevada](https://idioms.thefreedictionary.com/what+happens+in+Vegas+stays+in+Vegas)). 13 | 14 | `checkFieldNames`...FIXME 15 | 16 | --- 17 | 18 | `checkFieldNames` is used when: 19 | 20 | * `DDLUtils` is requested to [checkDataColNames](DDLUtils.md#checkDataColNames) 21 | * `FileFormatWriter` is requested to [write data out](../files/FileFormatWriter.md#write) 22 | -------------------------------------------------------------------------------- /docs/connectors/index.md: -------------------------------------------------------------------------------- 1 | # Connectors (Data Sources) 2 | 3 | The following are some of the built-in connectors (_data sources_) in Spark SQL based on the modern [Connector API](../connector/index.md) for various data formats and storages: 4 | 5 | * [kafka](../kafka/index.md) 6 | * [parquet](../parquet/index.md) 7 | * _others_ (see the menu on the left) 8 | -------------------------------------------------------------------------------- /docs/cost-based-optimization/EstimationUtils.md: -------------------------------------------------------------------------------- 1 | # EstimationUtils 2 | -------------------------------------------------------------------------------- /docs/dataset/.pages: -------------------------------------------------------------------------------- 1 | title: Dataset API 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/default-columns/.pages: -------------------------------------------------------------------------------- 1 | title: Default Columns 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/demo/.pages: -------------------------------------------------------------------------------- 1 | title: Demo 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/demo/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | --- 5 | 6 | # Demos 7 | -------------------------------------------------------------------------------- /docs/developer-api.md: -------------------------------------------------------------------------------- 1 | # Developer API 2 | 3 | [TAGS] 4 | -------------------------------------------------------------------------------- /docs/direct-queries-on-files/.pages: -------------------------------------------------------------------------------- 1 | title: Direct Queries on Files 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/direct-queries-on-files/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | --- 5 | 6 | # Direct Queries on Files 7 | 8 | **Direct Queries on Files** is enabled using [spark.sql.runSQLOnFiles](../configuration-properties.md#spark.sql.runSQLOnFiles) configuration property. 9 | 10 | Direct Queries on Files uses [ResolveSQLOnFile](../logical-analysis-rules/ResolveSQLOnFile.md) logical analysis rule to resolve queries. 11 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/.pages: -------------------------------------------------------------------------------- 1 | title: Execution Planning Strategies 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/SpecialLimits.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SpecialLimits 3 | --- 4 | 5 | # SpecialLimits Execution Planning Strategy 6 | 7 | `SpecialLimits` is an [execution planning strategy](SparkStrategy.md) that [Spark Planner](../SparkPlanner.md) uses to <>. 8 | 9 | === [[apply]] Applying SpecialLimits Strategy to Logical Plan (Executing SpecialLimits) -- `apply` Method 10 | 11 | [source, scala] 12 | ---- 13 | apply(plan: LogicalPlan): Seq[SparkPlan] 14 | ---- 15 | 16 | `apply`...FIXME 17 | 18 | `apply` is part of [GenericStrategy](../catalyst/GenericStrategy.md#apply) abstraction. 19 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/Window.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: Window 3 | --- 4 | 5 | # Window Execution Planning Strategy 6 | 7 | `Window` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/execution-planning-strategies/index.md: -------------------------------------------------------------------------------- 1 | # Execution Planning Strategies 2 | -------------------------------------------------------------------------------- /docs/expressions/AggregateWindowFunction.md: -------------------------------------------------------------------------------- 1 | # AggregateWindowFunction Expressions 2 | 3 | `AggregateWindowFunction` is an extension of the [DeclarativeAggregate](DeclarativeAggregate.md) and [WindowFunction](WindowFunction.md) abstractions for [aggregate window function expressions](#implementations). 4 | 5 | ## Implementations 6 | 7 | * `NthValue` 8 | * `RankLike` 9 | * [RowNumberLike](RowNumberLike.md) 10 | * `SizeBasedWindowFunction` 11 | 12 | ## Frame { #frame } 13 | 14 | ??? note "WindowFunction" 15 | 16 | ```scala 17 | frame: WindowFrame 18 | ``` 19 | 20 | `frame` is part of the [WindowFunction](WindowFunction.md#frame) abstraction. 21 | 22 | `frame` is a `SpecifiedWindowFrame` with the following: 23 | 24 | * `RowFrame` type 25 | * `UnboundedPreceding` lower expression 26 | * `CurrentRow` lower expression 27 | -------------------------------------------------------------------------------- /docs/expressions/ArrayBasedSimpleHigherOrderFunction.md: -------------------------------------------------------------------------------- 1 | # ArrayBasedSimpleHigherOrderFunction 2 | 3 | `ArrayBasedSimpleHigherOrderFunction` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/BasePredicate.md: -------------------------------------------------------------------------------- 1 | # BasePredicate Expressions 2 | 3 | `BasePredicate` is an [abstraction](#contract) of [predicate expressions](#implementations) that can be [evaluated](#eval) to a `Boolean` value. 4 | 5 | `BasePredicate` is created using [Predicate.create](Predicate.md#create) utility. 6 | 7 | ## Contract 8 | 9 | ### Evaluating 10 | 11 | ```scala 12 | eval( 13 | r: InternalRow): Boolean 14 | ``` 15 | 16 | ### Initializing 17 | 18 | ```scala 19 | initialize( 20 | partitionIndex: Int): Unit 21 | ``` 22 | 23 | ## Implementations 24 | 25 | * `InterpretedPredicate` 26 | -------------------------------------------------------------------------------- /docs/expressions/BinaryComparison.md: -------------------------------------------------------------------------------- 1 | # BinaryComparison Expressions 2 | 3 | `BinaryComparison` is an extension of the `BinaryOperator` and [Predicate](Predicate.md) abstractions for [comparison expressions](#implementations). 4 | 5 | ## Implementations 6 | 7 | * [EqualNullSafe](EqualNullSafe.md) 8 | * [EqualTo](EqualTo.md) 9 | * `GreaterThan` 10 | * `GreaterThanOrEqual` 11 | * `LessThan` 12 | * [LessThanOrEqual](LessThanOrEqual.md) 13 | -------------------------------------------------------------------------------- /docs/expressions/BinaryOperator.md: -------------------------------------------------------------------------------- 1 | # BinaryOperator 2 | 3 | `BinaryOperator` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/Collect.md: -------------------------------------------------------------------------------- 1 | # Collect Expressions 2 | 3 | `Collect` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/Count.md: -------------------------------------------------------------------------------- 1 | # Count Expression 2 | 3 | `Count` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/DecodeUsingSerializer.md: -------------------------------------------------------------------------------- 1 | # DecodeUsingSerializer 2 | 3 | `DecodeUsingSerializer` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/EncodeUsingSerializer.md: -------------------------------------------------------------------------------- 1 | # EncodeUsingSerializer 2 | 3 | `EncodeUsingSerializer` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/HigherOrderFunction.md: -------------------------------------------------------------------------------- 1 | # HigherOrderFunction 2 | 3 | `HigherOrderFunction` is an [extension](#contract) of the [Expression](Expression.md) abstraction for [FIXME](#implementations) that [method](#method) and...FIXME. 4 | 5 | ## Contract 6 | 7 | ### arguments 8 | 9 | ```scala 10 | arguments: Seq[Expression] 11 | ``` 12 | 13 | Used when...FIXME 14 | 15 | ### argumentTypes 16 | 17 | ```scala 18 | argumentTypes: Seq[AbstractDataType] 19 | ``` 20 | 21 | Used when...FIXME 22 | 23 | ### bind 24 | 25 | ```scala 26 | bind( 27 | f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): HigherOrderFunction 28 | ``` 29 | 30 | Used when...FIXME 31 | 32 | ### functions 33 | 34 | ```scala 35 | functions: Seq[Expression] 36 | ``` 37 | 38 | Used when...FIXME 39 | 40 | ### functionTypes 41 | 42 | ```scala 43 | functionTypes: Seq[AbstractDataType] 44 | ``` 45 | 46 | Used when...FIXME 47 | 48 | ## Implementations 49 | 50 | * `ArrayAggregate` 51 | * `MapZipWith` 52 | * [SimpleHigherOrderFunction](SimpleHigherOrderFunction.md) 53 | * `ZipWith` 54 | -------------------------------------------------------------------------------- /docs/expressions/InSet.md: -------------------------------------------------------------------------------- 1 | # InSet 2 | 3 | `InSet` is a [predicate expression](Expression.md#Predicate) that is an optimized variant of the [In](In.md) predicate expression. 4 | -------------------------------------------------------------------------------- /docs/expressions/ListQuery.md: -------------------------------------------------------------------------------- 1 | # ListQuery 2 | 3 | `ListQuery` is a [SubqueryExpression](SubqueryExpression.md) that represents SQL's [IN predicate with a subquery](../sql/AstBuilder.md#withPredicate). 4 | -------------------------------------------------------------------------------- /docs/expressions/Literal.md: -------------------------------------------------------------------------------- 1 | # Literal 2 | 3 | `Literal` is a [leaf expression](Expression.md#LeafExpression) to represent a Scala [value](#value) of a [specific type](#dataType). 4 | -------------------------------------------------------------------------------- /docs/expressions/MapBasedSimpleHigherOrderFunction.md: -------------------------------------------------------------------------------- 1 | # MapBasedSimpleHigherOrderFunction 2 | 3 | `MapBasedSimpleHigherOrderFunction` is...FIXME -------------------------------------------------------------------------------- /docs/expressions/MaxBy.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: MaxBy 3 | --- 4 | 5 | # MaxBy Expression 6 | 7 | `MaxBy` is a `MaxMinBy` aggregate function expression. 8 | -------------------------------------------------------------------------------- /docs/expressions/MutableProjection.md: -------------------------------------------------------------------------------- 1 | # MutableProjection Expression 2 | 3 | `MutableProjection` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/ParseToDate.md: -------------------------------------------------------------------------------- 1 | # ParseToDate 2 | 3 | `ParseToDate` is a [RuntimeReplaceable](RuntimeReplaceable.md) expression to represent [to_date](../standard-functions//datetime.md#to_date) function (in logical query plans). 4 | 5 | As a `RuntimeReplaceable` expression, `ParseToDate` is replaced by [Logical Query Optimizer](../catalyst/Optimizer.md#ReplaceExpressions) with the [child](#child) expression: 6 | 7 | * `Cast(left, DateType)` for `to_date(e: Column): Column` function 8 | 9 | * `Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType)` for `to_date(e: Column, fmt: String): Column` function 10 | -------------------------------------------------------------------------------- /docs/expressions/Predicate.md: -------------------------------------------------------------------------------- 1 | # Predicate Expressions 2 | 3 | `Predicate` is an extension of the [Expression](Expression.md) abstraction for [predicate expressions](#implementations) that evaluate to a value of [BooleanType](#dataType) type. 4 | 5 | ## Implementations 6 | 7 | * [BinaryComparison](BinaryComparison.md) 8 | * [Exists](Exists.md) 9 | * [In](In.md) 10 | * [InSet](InSet.md) 11 | * _others_ 12 | 13 | ## DataType 14 | 15 | ```scala 16 | dataType: DataType 17 | ``` 18 | 19 | `dataType` is part of the [Expression](Expression.md#dataType) abstraction. 20 | 21 | --- 22 | 23 | `dataType` is always [BooleanType](../types/DataType.md#BooleanType). 24 | 25 | ## Creating BasePredicate for Bound Expression 26 | 27 | ```scala 28 | create( 29 | e: Expression): BasePredicate 30 | create( 31 | e: Expression, 32 | inputSchema: Seq[Attribute]): BasePredicate 33 | ``` 34 | 35 | `create` [creates a BasePredicate](#createObject) for the given [Expression](Expression.md) that is [bound](#bindReference) to the input schema ([Attribute](Attribute.md)s). 36 | -------------------------------------------------------------------------------- /docs/expressions/Projection.md: -------------------------------------------------------------------------------- 1 | # Projection Functions 2 | 3 | `Projection` is an [abstraction](#contract) of [InternalRow converter functions](#implementations) to produce an [InternalRow](../InternalRow.md) for a given `InternalRow`. 4 | 5 | ```scala 6 | Projection: InternalRow => InternalRow 7 | ``` 8 | 9 | ## Contract 10 | 11 | ###  Initialization 12 | 13 | ```scala 14 | initialize( 15 | partitionIndex: Int): Unit 16 | ``` 17 | 18 | ## Implementations 19 | 20 | * `IdentityProjection` 21 | * [InterpretedProjection](InterpretedProjection.md) 22 | * `InterpretedSafeProjection` 23 | * `MutableProjection` 24 | * [UnsafeProjection](UnsafeProjection.md) 25 | -------------------------------------------------------------------------------- /docs/expressions/RowNumber.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RowNumber 3 | --- 4 | 5 | # RowNumber Aggregate Window Leaf Expression 6 | 7 | `RowNumber` is a [RowNumberLike](RowNumberLike.md) leaf expression known as [row_number](#prettyName). 8 | 9 | ## Evaluating Expression { #evaluateExpression } 10 | 11 | ??? note "DeclarativeAggregate" 12 | 13 | ```scala 14 | evaluateExpression: AttributeReference 15 | ``` 16 | 17 | `evaluateExpression` is part of the [DeclarativeAggregate](DeclarativeAggregate.md#evaluateExpression) abstraction. 18 | 19 | `evaluateExpression` is the [rowNumber](RowNumberLike.md#rowNumber) attribute reference. 20 | 21 | ## Pretty Name { #prettyName } 22 | 23 | ??? note "Expression" 24 | 25 | ```scala 26 | prettyName 27 | ``` 28 | 29 | `prettyName` is part of the [Expression](Expression.md#prettyName) abstraction. 30 | 31 | `prettyName` is the following text: 32 | 33 | ```text 34 | row_number 35 | ``` 36 | -------------------------------------------------------------------------------- /docs/expressions/RowOrdering.md: -------------------------------------------------------------------------------- 1 | # RowOrdering 2 | 3 | ## isOrderable 4 | 5 | ```scala 6 | isOrderable( 7 | dataType: DataType): Boolean 8 | isOrderable( 9 | exprs: Seq[Expression]): Boolean 10 | ``` 11 | 12 | `isOrderable` holds `true` when the [DataType](../types/DataType.md) is one of the following: 13 | 14 | * [NullType](../types/DataType.md#NullType) 15 | * [AtomicType](../types/AtomicType.md) 16 | * [StructType](../types/StructType.md) with all [fields](../types/StructType.md#fields) orderable (recursive) 17 | * [ArrayType](../types/ArrayType.md) with orderable type of the elements 18 | * [UserDefinedType](../types/UserDefinedType.md) 19 | 20 | `isOrderable` is used when: 21 | 22 | * [JoinSelection](../execution-planning-strategies/JoinSelection.md) execution planning strategy is executed (and [SortMergeJoinExec](../execution-planning-strategies/JoinSelection.md#createSortMergeJoin) is considered) 23 | * FIXME 24 | -------------------------------------------------------------------------------- /docs/expressions/SimpleHigherOrderFunction.md: -------------------------------------------------------------------------------- 1 | # SimpleHigherOrderFunction 2 | 3 | `SimpleHigherOrderFunction` is an [extension](#contract) of the [HigherOrderFunction](HigherOrderFunction.md) abstraction for [FIXME](#implementations) that [method](#method) and...FIXME. 4 | 5 | ## Contract 6 | 7 | ### argument 8 | 9 | ```scala 10 | argument: Expression 11 | ``` 12 | 13 | Used when...FIXME 14 | 15 | ### argumentType 16 | 17 | ```scala 18 | argumentType: AbstractDataType 19 | ``` 20 | 21 | Used when...FIXME 22 | 23 | ### function 24 | 25 | ```scala 26 | function: Expression 27 | ``` 28 | 29 | Used when...FIXME 30 | 31 | ## Implementations 32 | 33 | * [ArrayBasedSimpleHigherOrderFunction](ArrayBasedSimpleHigherOrderFunction.md) 34 | * [MapBasedSimpleHigherOrderFunction](MapBasedSimpleHigherOrderFunction.md) 35 | -------------------------------------------------------------------------------- /docs/expressions/SimpleTypedAggregateExpression.md: -------------------------------------------------------------------------------- 1 | # SimpleTypedAggregateExpression 2 | 3 | `SimpleTypedAggregateExpression` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/Stateful.md: -------------------------------------------------------------------------------- 1 | # Stateful 2 | 3 | `Stateful` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/expressions/UserDefinedExpression.md: -------------------------------------------------------------------------------- 1 | # UserDefinedExpression 2 | 3 | `UserDefinedExpression` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/file-based-data-scanning/index.md: -------------------------------------------------------------------------------- 1 | # File-Based Data Scanning 2 | 3 | Spark SQL uses [FileScanRDD](../rdds/FileScanRDD.md) for table scans of File-Based Data Sources (e.g., [parquet](../parquet/index.md)). 4 | 5 | The number of partitions in data scanning is based on the following: 6 | 7 | * [maxSplitBytes hint](../files/FilePartition.md#maxSplitBytes) 8 | * [Whether FileFormat is splitable or not](../files/FileFormat.md#isSplitable) 9 | * [Number of split files](../files/PartitionedFileUtil.md#splitFiles) 10 | * Bucket Pruning 11 | 12 | File-Based Data Scanning can be [bucketed or not](../physical-operators/FileSourceScanExec.md#bucketedScan). 13 | -------------------------------------------------------------------------------- /docs/files/AggregatePushDownUtils.md: -------------------------------------------------------------------------------- 1 | # AggregatePushDownUtils 2 | 3 | ## getSchemaForPushedAggregation 4 | 5 | ```scala 6 | getSchemaForPushedAggregation( 7 | aggregation: Aggregation, 8 | schema: StructType, 9 | partitionNames: Set[String], 10 | dataFilters: Seq[Expression]): Option[StructType] 11 | ``` 12 | 13 | `getSchemaForPushedAggregation`...FIXME 14 | 15 | --- 16 | 17 | `getSchemaForPushedAggregation` is used when: 18 | 19 | * `OrcScanBuilder` is requested to `pushAggregation` 20 | * `ParquetScanBuilder` is requested to [pushAggregation](../parquet/ParquetScanBuilder.md#pushAggregation) 21 | -------------------------------------------------------------------------------- /docs/files/BasicWriteTaskStats.md: -------------------------------------------------------------------------------- 1 | # BasicWriteTaskStats 2 | 3 | `BasicWriteTaskStats` is a [WriteTaskStats](WriteTaskStats.md). 4 | 5 | ## Creating Instance 6 | 7 | `BasicWriteTaskStats` takes the following to be created: 8 | 9 | * Partitions (`Seq[InternalRow]`) 10 | * Number of files 11 | * Number of bytes 12 | * Number of rows 13 | 14 | `BasicWriteTaskStats` is created when: 15 | 16 | * `BasicWriteTaskStatsTracker` is requested for [getFinalStats](BasicWriteTaskStatsTracker.md#getFinalStats) 17 | -------------------------------------------------------------------------------- /docs/files/DynamicPartitionDataConcurrentWriter.md: -------------------------------------------------------------------------------- 1 | # DynamicPartitionDataConcurrentWriter 2 | 3 | `DynamicPartitionDataConcurrentWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/files/DynamicPartitionDataSingleWriter.md: -------------------------------------------------------------------------------- 1 | # DynamicPartitionDataSingleWriter 2 | 3 | `DynamicPartitionDataSingleWriter` is a [BaseDynamicPartitionDataWriter](BaseDynamicPartitionDataWriter.md). 4 | 5 | ## Creating Instance 6 | 7 | `DynamicPartitionDataSingleWriter` takes the following to be created: 8 | 9 | * `WriteJobDescription` 10 | * `TaskAttemptContext` ([Apache Hadoop]({{ hadoop.api }}/org/apache/hadoop/mapreduce/TaskAttemptContext.html)) 11 | * `FileCommitProtocol` ([Spark Core]({{ book.spark_core }}/FileCommitProtocol)) 12 | * [SQLMetric](../SQLMetric.md)s 13 | 14 | `DynamicPartitionDataSingleWriter` is created when: 15 | 16 | * `FileFormatWriter` is requested to [write data out in a single Spark task](FileFormatWriter.md#executeTask) 17 | * `FileWriterFactory` is requested to [create a DataWriter](FileWriterFactory.md#createWriter) 18 | -------------------------------------------------------------------------------- /docs/files/FileScanBuilder.md: -------------------------------------------------------------------------------- 1 | # FileScanBuilder 2 | 3 | `FileScanBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/files/RecordReaderIterator.md: -------------------------------------------------------------------------------- 1 | # RecordReaderIterator 2 | 3 | [[creating-instance]] 4 | [[rowReader]] 5 | `RecordReaderIterator` is a Scala https://www.scala-lang.org/api/2.12.x/scala/collection/Iterator.html[scala.collection.Iterator] over the values of a Hadoop https://hadoop.apache.org/docs/r2.7.3/api/org/apache/hadoop/mapreduce/RecordReader.html[RecordReader]. 6 | 7 | `RecordReaderIterator` is <> when: 8 | 9 | * FIXME 10 | 11 | * [HadoopFileLinesReader](HadoopFileLinesReader.md#iterator) and `HadoopFileWholeTextReader` are requested for an value iterator 12 | 13 | * Legacy `OrcFileFormat` is requested to `buildReader` 14 | 15 | [[close]] 16 | When requested to close, `RecordReaderIterator` simply requests the underlying <> to close. 17 | 18 | [[hasNext]] 19 | When requested to check whether or not there more internal rows, `RecordReaderIterator` simply requests the underlying <> for `nextKeyValue`. 20 | 21 | [[next]] 22 | When requested for the next internal row, `RecordReaderIterator` simply requests the underlying <> for `getCurrentValue`. 23 | -------------------------------------------------------------------------------- /docs/files/SchemaMergeUtils.md: -------------------------------------------------------------------------------- 1 | # SchemaMergeUtils 2 | 3 | ## mergeSchemasInParallel 4 | 5 | ```scala 6 | mergeSchemasInParallel( 7 | sparkSession: SparkSession, 8 | parameters: Map[String, String], 9 | files: Seq[FileStatus], 10 | schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]): Option[StructType] 11 | ``` 12 | 13 | `mergeSchemasInParallel` determines a merged schema with a distributed Spark job. 14 | 15 | --- 16 | 17 | `mergeSchemasInParallel` creates an RDD with file paths and their lenght with the number of partitions up to the default parallelism (_number of CPU cores in a cluster_). 18 | 19 | In the end, `mergeSchemasInParallel` collects the RDD result that are [merged schemas](../types/StructType.md#merge) for files (per partition) that `mergeSchemasInParallel` merge all together to give the final merge schema. 20 | 21 | --- 22 | 23 | `mergeSchemasInParallel` is used when: 24 | 25 | * `OrcFileFormat` is requested to `inferSchema` 26 | * `OrcUtils` is requested to infer schema 27 | * `ParquetFileFormat` is requested to [infer schema](../parquet/ParquetFileFormat.md#inferSchema) 28 | -------------------------------------------------------------------------------- /docs/files/SharedInMemoryCache.md: -------------------------------------------------------------------------------- 1 | # SharedInMemoryCache 2 | 3 | `SharedInMemoryCache` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/files/WriteTaskStats.md: -------------------------------------------------------------------------------- 1 | # WriteTaskStats 2 | 3 | `WriteTaskStats` is the no-method contract of <> collected during a Write Task. 4 | 5 | [[implementations]] 6 | NOTE: [BasicWriteTaskStats](BasicWriteTaskStats.md) is the one and only known implementation of the <> in Apache Spark. 7 | -------------------------------------------------------------------------------- /docs/files/index.md: -------------------------------------------------------------------------------- 1 | # File-Based Connectors (Files) 2 | -------------------------------------------------------------------------------- /docs/generated-columns/.pages: -------------------------------------------------------------------------------- 1 | title: Generated Columns 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/generated-columns/GeneratedColumn.md: -------------------------------------------------------------------------------- 1 | # GeneratedColumn 2 | 3 | `GeneratedColumn` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/generated-columns/index.md: -------------------------------------------------------------------------------- 1 | # Generated Columns 2 | 3 | [Spark SQL 3.4.0]({{ spark.jira }}/SPARK-41290) comes with support for `GENERATED ALWAYS AS` syntax for defining **Generated Columns** in `CREATE TABLE` and `REPLACE TABLE` statements for data sources that support it. 4 | 5 | ``` sql hl_lines="3" 6 | CREATE TABLE default.example ( 7 | time TIMESTAMP, 8 | date DATE GENERATED ALWAYS AS (CAST(time AS DATE)) 9 | ) 10 | ``` 11 | 12 | Generated Columns are part of column definition. 13 | 14 | ``` antlr hl_lines="4" 15 | colDefinitionOption 16 | : NOT NULL 17 | | defaultExpression 18 | | generationExpression 19 | | commentSpec 20 | ; 21 | 22 | generationExpression 23 | : GENERATED ALWAYS AS '(' expression ')' 24 | ; 25 | ``` 26 | -------------------------------------------------------------------------------- /docs/hidden-file-metadata/.pages: -------------------------------------------------------------------------------- 1 | title: Hidden File Metadata 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/hints/.pages: -------------------------------------------------------------------------------- 1 | title: Hints (SQL) 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/hints/HintErrorHandler.md: -------------------------------------------------------------------------------- 1 | # HintErrorHandler 2 | 3 | `HintErrorHandler` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/hints/JoinHint.md: -------------------------------------------------------------------------------- 1 | # JoinHint 2 | 3 | `JoinHint` holds the [hints](HintInfo.md) of the [left](#leftHint) and [right](#rightHint) sides of a [Join](../logical-operators/Join.md#hint) logical operator. 4 | 5 | ## Creating Instance 6 | 7 | `JoinHint` takes the following to be created: 8 | 9 | * Left-Side Join [HintInfo](HintInfo.md) 10 | * Right-Side Join [HintInfo](HintInfo.md) 11 | 12 | `JoinHint` is created when: 13 | 14 | * [EliminateResolvedHint](../logical-optimizations/EliminateResolvedHint.md) logical optimization is executed (on [Join](../logical-operators/Join.md) logical operators with no hint associated explicitly) 15 | 16 | ## Text Representation 17 | 18 | ```scala 19 | toString: String 20 | ``` 21 | 22 | `toString` is part of the `Object` ([Java]({{ java.api }}/java/lang/Object.html#toString())) abstraction. 23 | 24 | `toString` returns the following (with the [leftHint](#leftHint) and [rightHint](#rightHint) if defined): 25 | 26 | ```text 27 | leftHint=[leftHint], rightHint=[rightHint] 28 | ``` 29 | -------------------------------------------------------------------------------- /docs/hints/join-strategy-hints.md: -------------------------------------------------------------------------------- 1 | # Join Strategy Hints 2 | 3 | **Join Strategy Hints** extend the existing BROADCAST join hint with other join strategy hints for shuffle-hash, sort-merge, cartesian-product. 4 | 5 | Join Strategy Hints were introduced to Apache Spark 3.0.0 as [SPARK-27225](https://issues.apache.org/jira/browse/SPARK-27225). 6 | 7 | Main abstractions: 8 | 9 | * [JoinStrategyHint](JoinStrategyHint.md) 10 | * [ResolveJoinStrategyHints](../logical-analysis-rules/ResolveJoinStrategyHints.md) logical resolution rule 11 | -------------------------------------------------------------------------------- /docs/hive/DataSinks.md: -------------------------------------------------------------------------------- 1 | # DataSinks 2 | 3 | CAUTION: FIXME 4 | -------------------------------------------------------------------------------- /docs/hive/DetermineTableStats.md: -------------------------------------------------------------------------------- 1 | # DetermineTableStats Logical PostHoc Resolution Rule -- Computing Total Size Table Statistic for HiveTableRelations 2 | 3 | `DetermineTableStats` is a HiveSessionStateBuilder.md#postHocResolutionRules[logical posthoc resolution rule] that the HiveSessionStateBuilder.md#analyzer[Hive-specific logical query plan analyzer] uses to <>. 4 | 5 | Technically, `DetermineTableStats` is a ../catalyst/Rule.md[Catalyst rule] for transforming ../spark-sql-LogicalPlan.md[logical plans], i.e. `Rule[LogicalPlan]`. 6 | 7 | === [[apply]] `apply` Method 8 | 9 | [source, scala] 10 | ---- 11 | apply(plan: LogicalPlan): LogicalPlan 12 | ---- 13 | 14 | NOTE: `apply` is part of ../catalyst/Rule.md#apply[Rule Contract] to apply a rule to a ../spark-sql-LogicalPlan.md[logical plan] (aka _execute a rule_). 15 | 16 | `apply`...FIXME 17 | -------------------------------------------------------------------------------- /docs/hive/HiveAnalysis.md: -------------------------------------------------------------------------------- 1 | # HiveAnalysis PostHoc Logical Resolution Rule 2 | 3 | `HiveAnalysis` is a HiveSessionStateBuilder.md#postHocResolutionRules[logical posthoc resolution rule] that the HiveSessionStateBuilder.md#analyzer[Hive-specific logical query plan analyzer] uses to <>. 4 | 5 | Technically, `HiveAnalysis` is a ../catalyst/Rule.md[Catalyst rule] for transforming ../spark-sql-LogicalPlan.md[logical plans], i.e. `Rule[LogicalPlan]`. 6 | -------------------------------------------------------------------------------- /docs/hive/ResolveHiveSerdeTable.md: -------------------------------------------------------------------------------- 1 | # ResolveHiveSerdeTable Logical Resolution Rule 2 | 3 | `ResolveHiveSerdeTable` is a logical resolution rule (i.e. `Rule[LogicalPlan]`) that the [Hive-specific logical query plan analyzer](HiveSessionStateBuilder.md#analyzer) uses to <>. 4 | 5 | `ResolveHiveSerdeTable` is part of [additional rules](../Analyzer.md#extendedResolutionRules) in [Resolution](../Analyzer.md#Resolution) fixed-point batch of rules. 6 | 7 | === [[apply]] Applying ResolveHiveSerdeTable Rule to Logical Plan -- `apply` Method 8 | 9 | [source, scala] 10 | ---- 11 | apply( 12 | plan: LogicalPlan): LogicalPlan 13 | ---- 14 | 15 | NOTE: `apply` is part of ../catalyst/Rule.md#apply[Rule Contract] to apply a rule to a ../spark-sql-LogicalPlan.md[logical plan]. 16 | 17 | `apply`...FIXME 18 | -------------------------------------------------------------------------------- /docs/hive/TableReader.md: -------------------------------------------------------------------------------- 1 | # TableReader 2 | 3 | `TableReader` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/images/AQEOptimizer.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/AQEOptimizer.png -------------------------------------------------------------------------------- /docs/images/DataWritingCommand-metrics.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/DataWritingCommand-metrics.png -------------------------------------------------------------------------------- /docs/images/Dataset.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/Dataset.png -------------------------------------------------------------------------------- /docs/images/HashAggregateExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/HashAggregateExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/KafkaOffsetReader-fetchSpecificOffsets.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/KafkaOffsetReader-fetchSpecificOffsets.png -------------------------------------------------------------------------------- /docs/images/ObjectHashAggregateExec-no-sort-fallback-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ObjectHashAggregateExec-no-sort-fallback-tasks.png -------------------------------------------------------------------------------- /docs/images/ObjectHashAggregateExec-sort-fallback-tasks-after-repartition.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ObjectHashAggregateExec-sort-fallback-tasks-after-repartition.png -------------------------------------------------------------------------------- /docs/images/ObjectHashAggregateExec-sort-fallback-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ObjectHashAggregateExec-sort-fallback-tasks.png -------------------------------------------------------------------------------- /docs/images/ObjectHashAggregateExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ObjectHashAggregateExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/QueryExecution-execution-pipeline.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/QueryExecution-execution-pipeline.png -------------------------------------------------------------------------------- /docs/images/ReorderJoin-createOrderedJoin-four-plans-after.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ReorderJoin-createOrderedJoin-four-plans-after.png -------------------------------------------------------------------------------- /docs/images/ReorderJoin-createOrderedJoin-four-plans-before.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ReorderJoin-createOrderedJoin-four-plans-before.png -------------------------------------------------------------------------------- /docs/images/ReorderJoin-createOrderedJoin-four-plans.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ReorderJoin-createOrderedJoin-four-plans.png -------------------------------------------------------------------------------- /docs/images/ShuffleExchangeExec-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/ShuffleExchangeExec-webui.png -------------------------------------------------------------------------------- /docs/images/SortAggregateExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/SortAggregateExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/SparkPlan-execute.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/SparkPlan-execute.png -------------------------------------------------------------------------------- /docs/images/SparkPlan-executeTake.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/SparkPlan-executeTake.png -------------------------------------------------------------------------------- /docs/images/demo-spill-webui-details-for-stage-tasks.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/demo-spill-webui-details-for-stage-tasks.png -------------------------------------------------------------------------------- /docs/images/demo-spill-webui-details-for-stage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/demo-spill-webui-details-for-stage.png -------------------------------------------------------------------------------- /docs/images/execute-pipeline.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/execute-pipeline.png -------------------------------------------------------------------------------- /docs/images/sort-spill.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/sort-spill.png -------------------------------------------------------------------------------- /docs/images/spark-sql-Analyzer.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-Analyzer.png -------------------------------------------------------------------------------- /docs/images/spark-sql-BroadcastExchangeExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-BroadcastExchangeExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/spark-sql-BroadcastHashJoinExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-BroadcastHashJoinExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-BroadcastNestedLoopJoinExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-BroadcastNestedLoopJoinExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/spark-sql-CatalogImpl-createExternalTable.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-CatalogImpl-createExternalTable.png -------------------------------------------------------------------------------- /docs/images/spark-sql-CatalogImpl.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-CatalogImpl.png -------------------------------------------------------------------------------- /docs/images/spark-sql-DataFrameWrite-insertInto-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-DataFrameWrite-insertInto-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-FileSourceScanExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-FileSourceScanExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-FilterExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-FilterExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/spark-sql-GenerateExec-doExecute.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-GenerateExec-doExecute.png -------------------------------------------------------------------------------- /docs/images/spark-sql-GenerateExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-GenerateExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/spark-sql-GlobalTempViewManager-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-GlobalTempViewManager-creating-instance.png -------------------------------------------------------------------------------- /docs/images/spark-sql-GlobalTempViewManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-GlobalTempViewManager.png -------------------------------------------------------------------------------- /docs/images/spark-sql-HiveExternalCatalog.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-HiveExternalCatalog.png -------------------------------------------------------------------------------- /docs/images/spark-sql-HiveMetastoreCatalog.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-HiveMetastoreCatalog.png -------------------------------------------------------------------------------- /docs/images/spark-sql-HiveSessionCatalog.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-HiveSessionCatalog.png -------------------------------------------------------------------------------- /docs/images/spark-sql-HiveSessionStateBuilder-SessionState.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-HiveSessionStateBuilder-SessionState.png -------------------------------------------------------------------------------- /docs/images/spark-sql-HiveSessionStateBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-HiveSessionStateBuilder.png -------------------------------------------------------------------------------- /docs/images/spark-sql-InMemoryTableScanExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-InMemoryTableScanExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-InputAdapter-doProduce.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-InputAdapter-doProduce.png -------------------------------------------------------------------------------- /docs/images/spark-sql-JDBCRelation-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-JDBCRelation-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-KafkaWriter-write-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-KafkaWriter-write-webui.png -------------------------------------------------------------------------------- /docs/images/spark-sql-LocalTableScanExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-LocalTableScanExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-RuntimeConfig.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-RuntimeConfig.png -------------------------------------------------------------------------------- /docs/images/spark-sql-ScalaUDAF-initialize.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-ScalaUDAF-initialize.png -------------------------------------------------------------------------------- /docs/images/spark-sql-ScalaUDAF-merge.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-ScalaUDAF-merge.png -------------------------------------------------------------------------------- /docs/images/spark-sql-ScalaUDAF-update.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-ScalaUDAF-update.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SessionCatalog.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SessionCatalog.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SessionState.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SessionState.png -------------------------------------------------------------------------------- /docs/images/spark-sql-ShuffledHashJoinExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-ShuffledHashJoinExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SortMergeJoinExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SortMergeJoinExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SparkOptimizer.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SparkOptimizer.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SparkSqlAstBuilder.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SparkSqlAstBuilder.png -------------------------------------------------------------------------------- /docs/images/spark-sql-SubqueryExec-webui-details-for-query.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-SubqueryExec-webui-details-for-query.png -------------------------------------------------------------------------------- /docs/images/spark-sql-UserDefinedAggregateFunction.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-UserDefinedAggregateFunction.png -------------------------------------------------------------------------------- /docs/images/spark-sql-WholeStageCodegenExec-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-WholeStageCodegenExec-webui.png -------------------------------------------------------------------------------- /docs/images/spark-sql-WindowExec-webui-query-details.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-WindowExec-webui-query-details.png -------------------------------------------------------------------------------- /docs/images/spark-sql-bucketing-sortmergejoin-bucketed-tables-no-exchanges.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-bucketing-sortmergejoin-bucketed-tables-no-exchanges.png -------------------------------------------------------------------------------- /docs/images/spark-sql-bucketing-sortmergejoin-filescans.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-bucketing-sortmergejoin-filescans.png -------------------------------------------------------------------------------- /docs/images/spark-sql-bucketing-sortmergejoin-one-bucketed-table.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-bucketing-sortmergejoin-one-bucketed-table.png -------------------------------------------------------------------------------- /docs/images/spark-sql-bucketing-sortmergejoin-sorted-dataset-and-bucketed-table.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-bucketing-sortmergejoin-sorted-dataset-and-bucketed-table.png -------------------------------------------------------------------------------- /docs/images/spark-sql-caching-webui-storage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-caching-webui-storage.png -------------------------------------------------------------------------------- /docs/images/spark-sql-performance-tuning-groupBy-aggregation-case1.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-performance-tuning-groupBy-aggregation-case1.png -------------------------------------------------------------------------------- /docs/images/spark-sql-performance-tuning-groupBy-aggregation-case4.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-performance-tuning-groupBy-aggregation-case4.png -------------------------------------------------------------------------------- /docs/images/spark-sql-pivot-webui-scanning-jobs.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-pivot-webui-scanning-jobs.png -------------------------------------------------------------------------------- /docs/images/spark-sql-pivot-webui.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-pivot-webui.png -------------------------------------------------------------------------------- /docs/images/spark-sql-tungsten-webui-storage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-sql-tungsten-webui-storage.png -------------------------------------------------------------------------------- /docs/images/spark-webui-storage.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/spark-webui-storage.png -------------------------------------------------------------------------------- /docs/images/webui-physical-plan.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/images/webui-physical-plan.png -------------------------------------------------------------------------------- /docs/jdbc/.pages: -------------------------------------------------------------------------------- 1 | title: JDBC 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/jdbc/AggregatedDialect.md: -------------------------------------------------------------------------------- 1 | # AggregatedDialect 2 | 3 | `AggregatedDialect` is a [JdbcDialect](JdbcDialect.md). 4 | 5 | ## Creating Instance 6 | 7 | `AggregatedDialect` takes the following to be created: 8 | 9 | * [JdbcDialect](JdbcDialect.md)s 10 | 11 | `AggregatedDialect` is created when: 12 | 13 | * `JdbcDialects` is requested for the [dialect](JdbcDialects.md#get) to handle a given URL (and there are two or more dialects) 14 | 15 | ## getTableExistsQuery { #getTableExistsQuery } 16 | 17 | ??? note "JdbcDialect" 18 | 19 | ```scala 20 | getTableExistsQuery( 21 | table: String): String 22 | ``` 23 | 24 | `getTableExistsQuery` is part of the [JdbcDialect](JdbcDialect.md#getTableExistsQuery) abstraction. 25 | 26 | `getTableExistsQuery` requests the first dialect (in the [dialects](#dialects)) to [getTableExistsQuery](JdbcDialect.md#getTableExistsQuery). 27 | -------------------------------------------------------------------------------- /docs/jdbc/JDBCScan.md: -------------------------------------------------------------------------------- 1 | # JDBCScan 2 | 3 | `JDBCScan` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/jdbc/JDBCScanBuilder.md: -------------------------------------------------------------------------------- 1 | # JDBCScanBuilder 2 | 3 | `JDBCScanBuilder` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/jdbc/JDBCTableCatalog.md: -------------------------------------------------------------------------------- 1 | # JDBCTableCatalog 2 | 3 | `JDBCTableCatalog` is a [TableCatalog](../connector/catalog/TableCatalog.md). 4 | 5 | `JDBCTableCatalog` is a [SupportsNamespaces](../connector/catalog/SupportsNamespaces.md). 6 | 7 | `JDBCTableCatalog` is a [FunctionCatalog](../connector/catalog/FunctionCatalog.md). 8 | 9 | ## tableExists { #tableExists } 10 | 11 | ??? note "TableCatalog" 12 | 13 | ```scala 14 | tableExists( 15 | ident: Identifier): Boolean 16 | ``` 17 | 18 | `tableExists` is part of the [TableCatalog](../connector/catalog/TableCatalog.md#tableExists) abstraction. 19 | 20 | `tableExists`...FIXME 21 | -------------------------------------------------------------------------------- /docs/jdbc/JdbcDialects.md: -------------------------------------------------------------------------------- 1 | --- 2 | tags: 3 | - DeveloperApi 4 | --- 5 | 6 | # JdbcDialects 7 | 8 | `JdbcDialects` is a registry of the supported [dialects](#dialects). 9 | 10 | ## Looking Up Dialect to Handle JDBC URL { #get } 11 | 12 | ```scala 13 | get( 14 | url: String): JdbcDialect 15 | ``` 16 | 17 | `get`...FIXME 18 | 19 | --- 20 | 21 | `get` is used when: 22 | 23 | * _many places_ 24 | -------------------------------------------------------------------------------- /docs/jdbc/JdbcUtils.md: -------------------------------------------------------------------------------- 1 | # JdbcUtils 2 | 3 | ## tableExists { #tableExists } 4 | 5 | ```scala 6 | tableExists( 7 | conn: Connection, 8 | options: JdbcOptionsInWrite): Boolean 9 | ``` 10 | 11 | `tableExists`...FIXME 12 | 13 | --- 14 | 15 | `tableExists` is used when: 16 | 17 | * `JdbcRelationProvider` is requested to [create a relation](JdbcRelationProvider.md#createRelation) 18 | * `JDBCTableCatalog` is requested to [tableExists](JDBCTableCatalog.md#tableExists) 19 | -------------------------------------------------------------------------------- /docs/jdbc/index.md: -------------------------------------------------------------------------------- 1 | # JDBC Connector 2 | 3 | Spark SQL supports loading data from tables using JDBC. 4 | 5 | Spark developers use [DataFrameReader.jdbc](../DataFrameReader.md#jdbc) to load data from an external table using JDBC. 6 | 7 | ```scala 8 | val table = spark.read.jdbc(url, table, properties) 9 | 10 | // Alternatively 11 | val table = spark.read.format("jdbc").options(...).load(...) 12 | ``` 13 | 14 | These one-liners create a [DataFrame](../DataFrame.md) that represents the distributed process of loading data from a database and a table (with additional properties). 15 | -------------------------------------------------------------------------------- /docs/kafka/InternalKafkaConsumer.md: -------------------------------------------------------------------------------- 1 | # InternalKafkaConsumer 2 | 3 | `InternalKafkaConsumer` is...FIXME 4 | 5 | === [[get]] Getting Single Kafka ConsumerRecord -- `get` Method 6 | 7 | [source, scala] 8 | ---- 9 | get( 10 | offset: Long, 11 | untilOffset: Long, 12 | pollTimeoutMs: Long, 13 | failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] 14 | ---- 15 | 16 | `get`...FIXME 17 | 18 | NOTE: `get` is used when...FIXME 19 | 20 | === [[getAvailableOffsetRange]] Getting Single AvailableOffsetRange -- `getAvailableOffsetRange` Method 21 | 22 | [source, scala] 23 | ---- 24 | getAvailableOffsetRange(): AvailableOffsetRange 25 | ---- 26 | 27 | `getAvailableOffsetRange`...FIXME 28 | 29 | NOTE: `getAvailableOffsetRange` is used when...FIXME 30 | -------------------------------------------------------------------------------- /docs/kafka/InternalKafkaProducerPool.md: -------------------------------------------------------------------------------- 1 | # InternalKafkaProducerPool 2 | 3 | ## spark.kafka.producer.cache.timeout 4 | 5 | `InternalKafkaProducerPool` uses [spark.kafka.producer.cache.timeout](configuration-properties.md#spark.kafka.producer.cache.timeout) when requested to [acquire a CachedKafkaProducer](#acquire). 6 | 7 | ## Acquiring CachedKafkaProducer 8 | 9 | ```scala 10 | acquire( 11 | kafkaParams: ju.Map[String, Object]): CachedKafkaProducer 12 | ``` 13 | 14 | `acquire`...FIXME 15 | 16 | `acquire` is used when...FIXME 17 | -------------------------------------------------------------------------------- /docs/kafka/KafkaBatch.md: -------------------------------------------------------------------------------- 1 | # KafkaBatch 2 | 3 | `KafkaBatch` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaBatchWrite.md: -------------------------------------------------------------------------------- 1 | # KafkaBatchWrite 2 | 3 | `KafkaBatchWrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaBatchWriterFactory.md: -------------------------------------------------------------------------------- 1 | # KafkaBatchWriterFactory 2 | 3 | `KafkaBatchWriterFactory` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaRecordToRowConverter.md: -------------------------------------------------------------------------------- 1 | # KafkaRecordToRowConverter 2 | 3 | ## kafkaSchema 4 | 5 | ```scala 6 | kafkaSchema( 7 | includeHeaders: Boolean): StructType 8 | ``` 9 | 10 | `kafkaSchema` is [schemaWithHeaders](#schemaWithHeaders) for the given `includeHeaders` enabled. Otherwise, `kafkaSchema` is [schemaWithoutHeaders](#schemaWithoutHeaders). 11 | 12 | --- 13 | 14 | `kafkaSchema` is used when: 15 | 16 | * `KafkaRelation` is requested for the [schema](KafkaRelation.md#schema) 17 | * `KafkaScan` is requested for the [readSchema](KafkaScan.md#readSchema) 18 | * `KafkaSource` ([Spark Structured Streaming]({{ book.structured_streaming }}/connectors/kafka/KafkaSource)) is requested for the `schema` 19 | * `KafkaSourceProvider` is requested for the [sourceSchema](KafkaSourceProvider.md#sourceSchema) 20 | * `KafkaTable` is requested for the [schema](KafkaTable.md#schema) 21 | -------------------------------------------------------------------------------- /docs/kafka/KafkaRowWriter.md: -------------------------------------------------------------------------------- 1 | # KafkaRowWriter 2 | 3 | `KafkaRowWriter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/KafkaSourceRDDPartition.md: -------------------------------------------------------------------------------- 1 | # KafkaSourceRDDPartition 2 | 3 | `KafkaSourceRDDPartition` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/kafka/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Connector 2 | 3 | **Kafka Connector** allows Spark SQL (and [Spark Structured Streaming]({{ book.structured_streaming }})) to read data from and write data to topics in Apache Kafka. 4 | 5 | Kafka Connector is available as [kafka](KafkaSourceProvider.md#shortName) format alias. 6 | 7 | The entry point is [KafkaSourceProvider](KafkaSourceProvider.md). 8 | 9 | !!! note 10 | **Apache Kafka** is a storage of records in a format-independent and fault-tolerant durable way. 11 | 12 | Learn more about Apache Kafka in the [official documentation](http://kafka.apache.org/documentation/) or [The Internals of Apache Kafka]({{ book.kafka }}). 13 | 14 | Kafka Connector supports [options](options.md) to fine-tune structured queries. 15 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/.pages: -------------------------------------------------------------------------------- 1 | title: Logical Analysis Rules 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/CTESubstitution.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CTESubstitution 3 | --- 4 | 5 | # CTESubstitution Logical Analysis Rule 6 | 7 | `CTESubstitution` is a [logical analysis rule](../Analyzer.md#batches) that [transforms a logical query plan](#apply) with...FIXME 8 | 9 | `CTESubstitution` is part of the [Substitution](../Analyzer.md#Substitution) fixed-point batch in the standard batches of the [Logical Analyzer](../Analyzer.md). 10 | 11 | `CTESubstitution` is a [Catalyst rule](../catalyst/Rule.md) for transforming [logical plans](../logical-operators/LogicalPlan.md) (`Rule[LogicalPlan]`). 12 | 13 | ## Executing Rule 14 | 15 | ```scala 16 | apply( 17 | plan: LogicalPlan): LogicalPlan 18 | ``` 19 | 20 | `apply`...FIXME 21 | 22 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 23 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/CleanupAliases.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CleanupAliases 3 | --- 4 | 5 | # CleanupAliases Logical Analysis Rule 6 | 7 | `CleanupAliases` is a [logical analysis rule](../Analyzer.md#batches) that <> with...FIXME 8 | 9 | `CleanupAliases` is part of the [Cleanup](../Analyzer.md#Cleanup) fixed-point batch in the standard batches of the [Logical Analyzer](../Analyzer.md). 10 | 11 | `CleanupAliases` is simply a [Catalyst rule](../catalyst/Rule.md) for transforming [logical plans](../logical-operators/LogicalPlan.md), i.e. `Rule[LogicalPlan]`. 12 | 13 | === [[apply]] Executing Rule -- `apply` Method 14 | 15 | [source, scala] 16 | ---- 17 | apply(plan: LogicalPlan): LogicalPlan 18 | ---- 19 | 20 | `apply`...FIXME 21 | 22 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 23 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/ResolveAggregateFunctions.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ResolveAggregateFunctions 3 | --- 4 | 5 | # ResolveAggregateFunctions Logical Analysis Rule 6 | 7 | `ResolveAggregateFunctions` is a [logical rule](../catalyst/Rule.md) (`Rule[LogicalPlan]`). 8 | 9 | `ResolveAggregateFunctions` is part of [Resolution](../Analyzer.md#Resolution) batch of [Logical Analyzer](../Analyzer.md). 10 | 11 | ## Creating Instance 12 | 13 | `ResolveAggregateFunctions` takes no arguments to be created. 14 | 15 | `ResolveAggregateFunctions` is created when: 16 | 17 | * `Analyzer` is requested for [batches](../Analyzer.md#batches) 18 | 19 | ## Executing Rule 20 | 21 | ```scala 22 | apply( 23 | plan: LogicalPlan): LogicalPlan 24 | ``` 25 | 26 | `apply` resolves the following operators in the input [LogicalPlan](../logical-operators/LogicalPlan.md): 27 | 28 | * [UnresolvedHaving](../logical-operators/UnresolvedHaving.md) with [Aggregate](../logical-operators/Aggregate.md) resolved 29 | * `Filter` with [Aggregate](../logical-operators/Aggregate.md) resolved 30 | * [Sort](../logical-operators/Sort.md) with [Aggregate](../logical-operators/Aggregate.md) resolved 31 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/ResolveCatalogs.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ResolveCatalogs 3 | --- 4 | 5 | # ResolveCatalogs Logical Resolution Rule 6 | 7 | `ResolveCatalogs` is a logical rule (`Rule[LogicalPlan]`). 8 | 9 | `ResolveCatalogs` is part of [Resolution](../Analyzer.md#Resolution) batch of [Logical Analyzer](../Analyzer.md). 10 | 11 | ## Creating Instance 12 | 13 | `ResolveCatalogs` takes the following to be created: 14 | 15 | * [CatalogManager](../connector/catalog/CatalogManager.md) 16 | 17 | `ResolveCatalogs` is created when: 18 | 19 | * `Analyzer` is requested for [batches](../Analyzer.md#batches) 20 | 21 | ## Executing Rule 22 | 23 | ```scala 24 | apply( 25 | plan: LogicalPlan): LogicalPlan 26 | ``` 27 | 28 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 29 | 30 | --- 31 | 32 | `apply`...FIXME 33 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/RewriteRowLevelCommand.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RewriteRowLevelCommand 3 | --- 4 | 5 | # RewriteRowLevelCommand Analysis Rule 6 | 7 | `RewriteRowLevelCommand` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/TableCapabilityCheck.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: TableCapabilityCheck 3 | --- 4 | 5 | # TableCapabilityCheck Extended Analysis Check 6 | 7 | `TableCapabilityCheck` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/logical-analysis-rules/index.md: -------------------------------------------------------------------------------- 1 | # Logical Analysis Rules 2 | 3 | **Logical Analysis Rule** is a [Rule](../catalyst/Rule.md) to operate on [logical query plans](../logical-operators/LogicalPlan.md) (`Rule[LogicalPlan]`). 4 | -------------------------------------------------------------------------------- /docs/logical-operators/.pages: -------------------------------------------------------------------------------- 1 | title: Logical Operators 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/logical-operators/AddColumns.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: AddColumns 3 | --- 4 | 5 | # AddColumns Logical Operator 6 | 7 | `AddColumns` is an `AlterTableCommand` logical operator that represents [ALTER TABLE ADD COLUMNS](../sql/AstBuilder.md#visitAddTableColumns) SQL statement (in a logical query plan). 8 | 9 | ## Creating Instance 10 | 11 | `AddColumns` takes the following to be created: 12 | 13 | * Table ([LogicalPlan](LogicalPlan.md)) 14 | * Columns to Add 15 | 16 | `AddColumns` is created when: 17 | 18 | * `AstBuilder` is requested to [parse ALTER TABLE ADD COLUMNS statement](../sql/AstBuilder.md#visitAddTableColumns) 19 | 20 | ## AlterTableAddColumnsCommand 21 | 22 | `AddColumns` is resolved to a [AlterTableAddColumnsCommand](AlterTableAddColumnsCommand.md) logical runnable command by [ResolveSessionCatalog](../logical-analysis-rules/ResolveSessionCatalog.md) logical resolution rule. 23 | -------------------------------------------------------------------------------- /docs/logical-operators/AnalyzeTablesCommand.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: AnalyzeTablesCommand 3 | --- 4 | 5 | # AnalyzeTablesCommand Logical Command 6 | 7 | `AnalyzeTablesCommand` is a [LeafRunnableCommand](LeafRunnableCommand.md) that represents `ANALYZE TABLES COMPUTE STATISTICS` SQL statement (`AnalyzeTables` logical command) at query execution. 8 | 9 | ## Creating Instance 10 | 11 | `AnalyzeTablesCommand` takes the following to be created: 12 | 13 | * Database Name 14 | * `noScan` flag 15 | 16 | `AnalyzeTablesCommand` is created when: 17 | 18 | * [ResolveSessionCatalog](../logical-analysis-rules/ResolveSessionCatalog.md) analysis rule is executed (and resolves an `AnalyzeTables` logical command for `ANALYZE TABLES COMPUTE STATISTICS` SQL statement) 19 | -------------------------------------------------------------------------------- /docs/logical-operators/BaseEvalPython.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: BaseEvalPython 3 | --- 4 | 5 | # BaseEvalPython Unary Logical Operators 6 | 7 | `BaseEvalPython` is an [extension](#contract) of the [UnaryNode](LogicalPlan.md#UnaryNode) abstraction for [unary logical operators](#implementations) that can execute [PythonUDFs](#udfs). 8 | 9 | ## Contract 10 | 11 | ### udfs 12 | 13 | ```scala 14 | udfs: Seq[PythonUDF] 15 | ``` 16 | 17 | [PythonUDF](../expressions/PythonUDF.md)s 18 | 19 | ### resultAttrs { #resultAttrs } 20 | 21 | ```scala 22 | resultAttrs: Seq[Attribute] 23 | ``` 24 | 25 | Result [Attribute](../expressions/Attribute.md)s 26 | 27 | Used when: 28 | 29 | * `BaseEvalPython` is requested for the [output](#output) and [producedAttributes](#producedAttributes) 30 | 31 | ## Implementations 32 | 33 | * [ArrowEvalPython](ArrowEvalPython.md) 34 | * `BatchEvalPython` 35 | -------------------------------------------------------------------------------- /docs/logical-operators/CTERelationDef.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CTERelationDef 3 | --- 4 | 5 | # CTERelationDef Unary Logical Operator 6 | 7 | `CTERelationDef` is a [unary logical operator](LogicalPlan.md#UnaryNode). 8 | 9 | ## Creating Instance 10 | 11 | `CTERelationDef` takes the following to be created: 12 | 13 | * Child [logical operator](LogicalPlan.md) 14 | * ID (default: a new unique ID) 15 | 16 | `CTERelationDef` is created when: 17 | 18 | * [CTESubstitution](../logical-analysis-rules/CTESubstitution.md) logical analysis rule is executed 19 | 20 | ## Node Patterns { #nodePatterns } 21 | 22 | ??? note "TreeNode" 23 | 24 | ```scala 25 | nodePatterns: Seq[TreePattern] 26 | ``` 27 | 28 | `nodePatterns` is part of the [TreeNode](../catalyst/TreeNode.md#nodePatterns) abstraction. 29 | 30 | `nodePatterns` is [CTE](../catalyst/TreePattern.md#CTE). 31 | -------------------------------------------------------------------------------- /docs/logical-operators/ClearCacheCommand.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ClearCacheCommand 3 | --- 4 | 5 | # ClearCacheCommand Logical Command 6 | 7 | `ClearCacheCommand` is a [logical command](RunnableCommand.md) to [remove all cached tables from the in-memory cache](../Catalog.md#clearCache). 8 | 9 | `ClearCacheCommand` corresponds to `CLEAR CACHE` SQL statement. 10 | 11 | ## clearCache Labeled Alternative { #clearCache } 12 | 13 | `ClearCacheCommand` is described by `clearCache` labeled alternative in `statement` expression in [SqlBaseParser.g4](../sql/AstBuilder.md#grammar) and parsed using [SparkSqlParser](../sql/SparkSqlParser.md). 14 | -------------------------------------------------------------------------------- /docs/logical-operators/CommentOnTable.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CommentOnTable 3 | --- 4 | 5 | # CommentOnTable Logical Command 6 | 7 | `CommentOnTable` is a [Command](Command.md) that represents [COMMENT ON TABLE](../sql/AstBuilder.md#visitCommentTable) SQL command. 8 | 9 | ## Creating Instance 10 | 11 | `CommentOnTable` takes the following to be created: 12 | 13 | * [LogicalPlan](LogicalPlan.md) 14 | * Comment 15 | 16 | `CommentOnTable` is created when: 17 | 18 | * `AstBuilder` is requested to [parse COMMENT ON TABLE command](../sql/AstBuilder.md#visitCommentTable) 19 | 20 | ## Execution Planning 21 | 22 | `CommentOnTable` is resolved to [AlterTableExec](../physical-operators/AlterTableExec.md) by [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy. 23 | -------------------------------------------------------------------------------- /docs/logical-operators/CreateTempViewUsing.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CreateTempViewUsing 3 | --- 4 | 5 | # CreateTempViewUsing Logical Command 6 | 7 | `CreateTempViewUsing` is a [LeafRunnableCommand](LeafRunnableCommand.md) that represents the following SQL statement at execution: 8 | 9 | ```sql 10 | CREATE (OR REPLACE)? GLOBAL? TEMPORARY VIEW 11 | tableIdentifier ('(' colTypeList ')')? 12 | USING multipartIdentifier 13 | (OPTIONS propertyList)? 14 | ``` 15 | 16 | ## Creating Instance 17 | 18 | `CreateTempViewUsing` takes the following to be created: 19 | 20 | * `TableIdentifier` 21 | * User-Specified Schema 22 | * `replace` flag 23 | * `global` flag 24 | * Provider Name 25 | * Options 26 | 27 | `CreateTempViewUsing` is created when: 28 | 29 | * `SparkSqlAstBuilder` is requested to parse [CREATE TEMPORARY TABLE USING](../sql/SparkSqlAstBuilder.md#visitCreateTable) (_deprecated_) and [CREATE TEMPORARY VIEW](../sql/SparkSqlAstBuilder.md#visitCreateTempViewUsing) statements 30 | -------------------------------------------------------------------------------- /docs/logical-operators/DeleteFromTable.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: DeleteFromTable 3 | --- 4 | 5 | # DeleteFromTable Logical Command 6 | 7 | `DeleteFromTable` is a [Command](Command.md) that represents [DELETE FROM](../sql/AstBuilder.md#visitDeleteFromTable) SQL statement. 8 | 9 | `DeleteFromTable` is a [SupportsSubquery](SupportsSubquery.md). 10 | 11 | ## Creating Instance 12 | 13 | `DeleteFromTable` takes the following to be created: 14 | 15 | * [LogicalPlan](LogicalPlan.md) of the table 16 | * Condition [Expression](../expressions/Expression.md) (optional) 17 | 18 | `DeleteFromTable` is created when: 19 | 20 | * `AstBuilder` is requested to [parse DELETE FROM SQL statement](../sql/AstBuilder.md#visitDeleteFromTable) 21 | 22 | ## Execution Planning 23 | 24 | `DeleteFromTable` command is resolved to [DeleteFromTableExec](../physical-operators/DeleteFromTableExec.md) physical operator by [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy. 25 | 26 | It is only supported for `DeleteFromTable` command over [DataSourceV2ScanRelation](DataSourceV2ScanRelation.md) relations (_v2 tables_). 27 | -------------------------------------------------------------------------------- /docs/logical-operators/ExposesMetadataColumns.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ExposesMetadataColumns 3 | --- 4 | 5 | # ExposesMetadataColumns Logical Operators 6 | 7 | `ExposesMetadataColumns` is an [extension](#contract) of the [LogicalPlan](LogicalPlan.md) abstraction for [logical operators](#implementations) that can [withMetadataColumns](#withMetadataColumns). 8 | 9 | ## Contract 10 | 11 | ### withMetadataColumns 12 | 13 | ```scala 14 | withMetadataColumns(): LogicalPlan 15 | ``` 16 | 17 | See: 18 | 19 | * [DataSourceV2Relation](DataSourceV2Relation.md#withMetadataColumns) 20 | * [LogicalRelation](LogicalRelation.md#withMetadataColumns) 21 | 22 | Used when: 23 | 24 | * [AddMetadataColumns](../logical-analysis-rules/AddMetadataColumns.md) logical analysis rule is executed (and [addMetadataCol](../logical-analysis-rules/AddMetadataColumns.md#addMetadataCol)) 25 | 26 | ## Implementations 27 | 28 | * [DataSourceV2Relation](DataSourceV2Relation.md) 29 | * [LogicalRelation](LogicalRelation.md) 30 | * `StreamingRelation` ([Spark Structured Streaming]({{ book.structured_streaming }}/logical-operators/StreamingRelation)) 31 | -------------------------------------------------------------------------------- /docs/logical-operators/IgnoreCachedData.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: IgnoreCachedData 3 | --- 4 | 5 | # IgnoreCachedData Logical Operators 6 | 7 | `IgnoreCachedData` is a marker interface for [logical operators](LogicalPlan.md) that should be skipped (_ignored_) by [CacheManager](../CacheManager.md) (while [replacing segments of a logical query with cached data](../CacheManager.md#useCachedData)). 8 | 9 | ## Implementations 10 | 11 | * [ClearCacheCommand](ClearCacheCommand.md) 12 | * `ResetCommand` 13 | -------------------------------------------------------------------------------- /docs/logical-operators/LoadDataCommand.md: -------------------------------------------------------------------------------- 1 | # LoadDataCommand 2 | 3 | `LoadDataCommand` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-operators/MultiInstanceRelation.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: MultiInstanceRelation 3 | --- 4 | 5 | # MultiInstanceRelation Logical Operators 6 | 7 | `MultiInstanceRelation` is an [abstraction](#contract) of [logical operators](#implementations) for which a single instance might appear multiple times in a logical query plan. 8 | 9 | ## Contract 10 | 11 | ### Creating New Instance { #newInstance } 12 | 13 | ```scala 14 | newInstance(): LogicalPlan 15 | ``` 16 | 17 | Used when: 18 | 19 | * [ResolveRelations](../logical-analysis-rules/ResolveRelations.md) logical resolution rule is executed 20 | * `DeduplicateRelations` logical resolution rule is executed 21 | 22 | ## Implementations 23 | 24 | * [CTERelationRef](CTERelationRef.md) 25 | * [DataSourceV2Relation](DataSourceV2Relation.md) 26 | * [ExternalRDD](ExternalRDD.md) 27 | * [HiveTableRelation](../hive/HiveTableRelation.md) 28 | * [InMemoryRelation](InMemoryRelation.md) 29 | * [LocalRelation](LocalRelation.md) 30 | * [LogicalRDD](LogicalRDD.md) 31 | * [LogicalRelation](LogicalRelation.md) 32 | * _others_ 33 | -------------------------------------------------------------------------------- /docs/logical-operators/NameParameterizedQuery.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: NameParameterizedQuery 3 | --- 4 | 5 | # NameParameterizedQuery Unary Logical Operator 6 | 7 | `NameParameterizedQuery` is a [ParameterizedQuery](ParameterizedQuery.md) logical operator that represents a parameterized query with named parameters. 8 | 9 | ## Creating Instance 10 | 11 | `NameParameterizedQuery` takes the following to be created: 12 | 13 | * Child [LogicalPlan](LogicalPlan.md) 14 | * Arguments (`Map[String, Expression]`) 15 | 16 | `NameParameterizedQuery` is created when: 17 | 18 | * `SparkSession` is requested to [sql](../SparkSession.md#sql) 19 | -------------------------------------------------------------------------------- /docs/logical-operators/OrderPreservingUnaryNode.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: OrderPreservingUnaryNode 3 | --- 4 | 5 | # OrderPreservingUnaryNode Logical Operators 6 | 7 | `OrderPreservingUnaryNode` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/logical-operators/ParsedStatement.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ParsedStatement 3 | --- 4 | 5 | # ParsedStatement Logical Operators 6 | 7 | `ParsedStatement` is an extension of the [LogicalPlan](LogicalPlan.md) abstraction for [logical operators](#implementations) that hold exactly what was parsed from SQL statements. 8 | 9 | `ParsedStatement` are never resolved and must be converted to concrete logical plans. 10 | 11 | ## Implementations 12 | 13 | * `LeafParsedStatement` 14 | * `UnaryParsedStatement` 15 | * [InsertIntoStatement](InsertIntoStatement.md) 16 | -------------------------------------------------------------------------------- /docs/logical-operators/ReplaceData.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ReplaceData 3 | --- 4 | 5 | # ReplaceData Logical Operator 6 | 7 | `ReplaceData` is a [RowLevelWrite](RowLevelWrite.md) logical operator. 8 | 9 | `ReplaceData` represents [DataSourceV2Relation](DataSourceV2Relation.md) with [SupportsRowLevelOperations](../connector/SupportsRowLevelOperations.md) in [DeleteFromTable](DeleteFromTable.md) operators. 10 | 11 | ## Creating Instance 12 | 13 | `ReplaceData` takes the following to be created: 14 | 15 | * [NamedRelation](NamedRelation.md) of the target table 16 | * Condition [Expression](../expressions/Expression.md) 17 | * Query [LogicalPlan](LogicalPlan.md) 18 | * [NamedRelation](NamedRelation.md) of the original table 19 | * Optional [Write](../connector/Write.md) (default: undefined) 20 | 21 | `ReplaceData` is created when: 22 | 23 | * [RewriteDeleteFromTable](../logical-analysis-rules/RewriteDeleteFromTable.md) analysis rule is executed (to [buildReplaceDataPlan](../logical-analysis-rules/RewriteDeleteFromTable.md#buildReplaceDataPlan)) 24 | -------------------------------------------------------------------------------- /docs/logical-operators/ResolvedHint.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ResolvedHint 3 | --- 4 | 5 | # ResolvedHint Unary Logical Operator 6 | 7 | `ResolvedHint` is a [unary logical operator](LogicalPlan.md#UnaryNode) to represent resolved hint nodes in a [logical query plan](#child). 8 | 9 | ## Creating Instance 10 | 11 | `ResolvedHint` takes the following to be created: 12 | 13 | * Child [LogicalPlan](LogicalPlan.md) 14 | * [HintInfo](../hints/HintInfo.md) 15 | 16 | `ResolvedHint` is created when: 17 | 18 | * [ResolveJoinStrategyHints](../logical-analysis-rules/ResolveJoinStrategyHints.md) logical resolution rule is [executed](../logical-analysis-rules/ResolveJoinStrategyHints.md#applyJoinStrategyHint) 19 | * [broadcast](../standard-functions/index.md#broadcast) standard function is used (on a `Dataset`) 20 | * `CacheManager` is requested to [useCachedData](../CacheManager.md#useCachedData) 21 | 22 | ## Query Execution Planning 23 | 24 | [BasicOperators](../execution-planning-strategies/BasicOperators.md) execution planning strategy throws an `IllegalStateException` for `ResolvedHint`s when executed. 25 | -------------------------------------------------------------------------------- /docs/logical-operators/ResolvedTable.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ResolvedTable 3 | --- 4 | 5 | # ResolvedTable Leaf Logical Operator 6 | 7 | `ResolvedTable` is a [leaf logical operator](LeafNode.md). 8 | 9 | ## Creating Instance 10 | 11 | `ResolvedTable` takes the following to be created: 12 | 13 | * [TableCatalog](../connector/catalog/TableCatalog.md) 14 | * `Identifier` 15 | * [Table](../connector/Table.md) 16 | 17 | `ResolvedTable` is created when: 18 | 19 | * `ResolveTables` logical resolution rule is executed (for [UnresolvedTable](UnresolvedTable.md) and [UnresolvedTableOrView](UnresolvedTableOrView.md)) 20 | * [ResolveRelations](../logical-analysis-rules/ResolveRelations.md) logical resolution rule is executed ([lookupTableOrView](../logical-analysis-rules/ResolveRelations.md#lookupTableOrView)) 21 | * [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (for `RenameTable`) 22 | -------------------------------------------------------------------------------- /docs/logical-operators/RowLevelWrite.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RowLevelWrite 3 | --- 4 | 5 | # RowLevelWrite Logical Operators 6 | 7 | `RowLevelWrite` is an [extension](#contract) of the [V2WriteCommand](V2WriteCommand.md) abstraction for [write commands](#implementations) with [SupportsSubquery](SupportsSubquery.md). 8 | 9 | ## Contract (Subset) 10 | 11 | ### operation 12 | 13 | ```scala 14 | operation: RowLevelOperation 15 | ``` 16 | 17 | See: 18 | 19 | * [ReplaceData](ReplaceData.md#operation) 20 | * [WriteDelta](WriteDelta.md#operation) 21 | 22 | Used when: 23 | 24 | * `GroupBasedRowLevelOperation` is requested to `unapply` 25 | * `WriteDelta` is requested to [metadataAttrsResolved](WriteDelta.md#metadataAttrsResolved), [rowIdAttrsResolved](WriteDelta.md#rowIdAttrsResolved) 26 | * `GroupBasedRowLevelOperationScanPlanning` is requested to [apply](../logical-optimizations/GroupBasedRowLevelOperationScanPlanning.md#apply) 27 | * `RewrittenRowLevelCommand` is requested to `unapply` 28 | * `RowLevelOperationRuntimeGroupFiltering` is requested to `apply` 29 | 30 | ## Implementations 31 | 32 | * [ReplaceData](ReplaceData.md) 33 | * [WriteDelta](WriteDelta.md) 34 | -------------------------------------------------------------------------------- /docs/logical-operators/SetCatalogAndNamespace.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SetCatalogAndNamespace 3 | --- 4 | 5 | # SetCatalogAndNamespace Logical Command 6 | 7 | `SetCatalogAndNamespace` is a `UnaryCommand` that represents [USE SQL statement](../sql/AstBuilder.md#visitUse). 8 | 9 | ## Creating Instance 10 | 11 | `SetCatalogAndNamespace` takes the following to be created: 12 | 13 | * Child [LogicalPlan](LogicalPlan.md) 14 | 15 | `SetCatalogAndNamespace` is created when: 16 | 17 | * `AstBuilder` is requested to [parse USE statement](../sql/AstBuilder.md#visitUse) 18 | 19 | ## Execution Planning 20 | 21 | `SetCatalogAndNamespace` is resolved to [SetCatalogAndNamespaceExec](../physical-operators/SetCatalogAndNamespaceExec.md) physical command by [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy. 22 | -------------------------------------------------------------------------------- /docs/logical-operators/ShowColumnsCommand.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ShowColumnsCommand 3 | --- 4 | 5 | # ShowColumnsCommand Leaf Logical Command 6 | 7 | `ShowColumnsCommand` is a [LeafRunnableCommand](LeafRunnableCommand.md) that represents [ShowColumns](ShowColumns.md) logical command at logical query resolution phase. 8 | 9 | ```sql 10 | SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; 11 | ``` 12 | 13 | ## Creating Instance 14 | 15 | `ShowColumnsCommand` takes the following to be created: 16 | 17 | * Database Name 18 | * Table Name 19 | * Output [Attribute](../expressions/Attribute.md)s 20 | 21 | `ShowColumnsCommand` is created when: 22 | 23 | * `ResolveSessionCatalog` logical resolution rule is requested to [resolve ShowColumns logical operator](../logical-analysis-rules/ResolveSessionCatalog.md#ShowColumns) 24 | -------------------------------------------------------------------------------- /docs/logical-operators/ShowTables.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ShowTables 3 | --- 4 | 5 | # ShowTables Logical Command 6 | 7 | `ShowTables` is a [logical command](Command.md) that represents [SHOW TABLES](../sql/AstBuilder.md#visitShowTables) SQL statement. 8 | 9 | ```text 10 | SHOW TABLES ((FROM | IN) multipartIdentifier)? 11 | (LIKE? pattern=STRING)? 12 | ``` 13 | 14 | !!! note 15 | `ShowTables` is resolved to [ShowTablesExec](../physical-operators/ShowTablesExec.md) physical command. 16 | 17 | ## Creating Instance 18 | 19 | `ShowTables` takes the following to be created: 20 | 21 | * [Logical Operator](LogicalPlan.md) 22 | * Optional Pattern (of tables to show) 23 | 24 | `ShowTables` is created when `AstBuilder` is requested to [visitShowTables](../sql/AstBuilder.md#visitShowTables). 25 | 26 | ## Output Attributes 27 | 28 | ```scala 29 | output: Seq[Attribute] 30 | ``` 31 | 32 | `output` is the following [attributes](../expressions/Attribute.md): 33 | 34 | * namespace 35 | * tableName 36 | 37 | `output` is part of the [Command](Command.md#output) abstraction. 38 | -------------------------------------------------------------------------------- /docs/logical-operators/SupportsSubquery.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: SupportsSubquery 3 | --- 4 | 5 | # SupportsSubquery Logical Operators 6 | 7 | `SupportsSubquery` is a marker interface (and an extension of the [LogicalPlan](LogicalPlan.md) abstraction) for [logical operators](#implementations) that support subqueries. 8 | 9 | `SupportsSubquery` is resolved by [ResolveSubquery](../logical-analysis-rules/ResolveSubquery.md) logical resolution rule. 10 | 11 | ## Implementations 12 | 13 | * [DeleteFromTable](DeleteFromTable.md) 14 | * [MergeIntoTable](MergeIntoTable.md) 15 | * [UpdateTable](UpdateTable.md) 16 | -------------------------------------------------------------------------------- /docs/logical-operators/V2CreateTablePlan.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: V2CreateTablePlan 3 | --- 4 | 5 | # V2CreateTablePlan Logical Operators 6 | 7 | `V2CreateTablePlan` is an [extension](#contract) of the [LogicalPlan](LogicalPlan.md) abstraction for [logical operators](#implementations) that create or replace V2 table definitions. 8 | 9 | ## Contract 10 | 11 | ### Partitioning Transforms 12 | 13 | ```scala 14 | partitioning: Seq[Transform] 15 | ``` 16 | 17 | Partitioning [Transform](../connector/Transform.md)s 18 | 19 | ### tableName 20 | 21 | ```scala 22 | tableName: Identifier 23 | ``` 24 | 25 | Used when [PreprocessTableCreation](../logical-analysis-rules/PreprocessTableCreation.md) post-hoc logical resolution rule is executed 26 | 27 | ### Table Schema 28 | 29 | ```scala 30 | tableSchema: StructType 31 | ``` 32 | 33 | ### withPartitioning 34 | 35 | ```scala 36 | withPartitioning( 37 | rewritten: Seq[Transform]): V2CreateTablePlan 38 | ``` 39 | 40 | ## Implementations 41 | 42 | * [CreateTableAsSelect](CreateTableAsSelect.md) 43 | * `ReplaceTable` 44 | * `ReplaceTableAsSelect` 45 | -------------------------------------------------------------------------------- /docs/logical-operators/WriteFiles.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: WriteFiles 3 | --- 4 | 5 | # WriteFiles Unary Logical Operator 6 | 7 | `WriteFiles` is a [unary logical operator](LogicalPlan.md#UnaryNode). 8 | 9 | ## Creating Instance 10 | 11 | `WriteFiles` takes the following to be created: 12 | 13 | * Child [LogicalPlan](LogicalPlan.md) 14 | * [FileFormat](../files/FileFormat.md) 15 | * Partition Columns ([Attribute](../expressions/Attribute.md)s) 16 | * [BucketSpec](../bucketing/BucketSpec.md) 17 | * Options 18 | * Static Partitions (`TablePartitionSpec`) 19 | 20 | `WriteFiles` is created when: 21 | 22 | * `V1Writes` logical optimization is executed 23 | 24 | ## Query Execution 25 | 26 | `WriteFiles` is planned as [WriteFilesExec](../physical-operators/WriteFilesExec.md) physical operator by [BasicOperators](../execution-planning-strategies/BasicOperators.md) execution planning strategy. 27 | -------------------------------------------------------------------------------- /docs/logical-operators/index.md: -------------------------------------------------------------------------------- 1 | # Logical Operators 2 | 3 | **Logical Operators** (_Logical Relational Operators_) are building blocks of logical query plans in Spark SQL. 4 | 5 | **Logical Query Plan** is a tree of [nodes](../catalyst/TreeNode.md) of logical operators that in turn can have (trees of) [Catalyst expressions](../expressions/Expression.md). In other words, there are _at least_ two trees at every level (operator). 6 | 7 | The main abstraction is [LogicalPlan](LogicalPlan.md) that is a recursive data structure with zero, one, two or more child logical operators: 8 | 9 | * [LeafNode](LeafNode.md) 10 | * [UnaryNode](LogicalPlan.md#UnaryNode) 11 | * [BinaryNode](LogicalPlan.md#BinaryNode) 12 | 13 | Among the logical operators are [Command](Command.md)s. 14 | -------------------------------------------------------------------------------- /docs/logical-optimizations/.pages: -------------------------------------------------------------------------------- 1 | title: Logical Optimizations 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/logical-optimizations/AQEPropagateEmptyRelation.md: -------------------------------------------------------------------------------- 1 | # AQEPropagateEmptyRelation Adaptive Logical Optimization 2 | 3 | `AQEPropagateEmptyRelation` is a logical optimization in [Adaptive Query Execution](../adaptive-query-execution/index.md). 4 | 5 | `AQEPropagateEmptyRelation` is a [Catalyst rule](../catalyst/Rule.md) for transforming [logical plans](../logical-operators/LogicalPlan.md) (`Rule[LogicalPlan]`). 6 | 7 | ## Creating Instance 8 | 9 | `AQEPropagateEmptyRelation` takes no arguments to be created. 10 | 11 | `AQEPropagateEmptyRelation` is created when: 12 | 13 | * `AQEOptimizer` is requested for the [default batches](../adaptive-query-execution/AQEOptimizer.md#defaultBatches) (of adaptive optimizations) 14 | 15 | ## Executing Rule 16 | 17 | ```scala 18 | apply( 19 | plan: LogicalPlan): LogicalPlan 20 | ``` 21 | 22 | `apply`...FIXME 23 | 24 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 25 | -------------------------------------------------------------------------------- /docs/logical-optimizations/CollapseWindow.md: -------------------------------------------------------------------------------- 1 | # CollapseWindow Logical Optimization 2 | 3 | `CollapseWindow` is a [base logical optimization](../catalyst/Optimizer.md#batches) that <>. 4 | 5 | `CollapseWindow` is part of the [Operator Optimization](../catalyst/Optimizer.md#Operator_Optimization) fixed-point batch in the standard batches of the [Logical Optimizer](../catalyst/Optimizer.md). 6 | 7 | `CollapseWindow` is simply a <> for transforming <>, i.e. `Rule[LogicalPlan]`. 8 | 9 | [source, scala] 10 | ---- 11 | // FIXME: DEMO 12 | import org.apache.spark.sql.catalyst.optimizer.CollapseWindow 13 | 14 | val logicalPlan = ??? 15 | val afterCollapseWindow = CollapseWindow(logicalPlan) 16 | ---- 17 | 18 | ## Executing Rule 19 | 20 | ```scala 21 | apply(plan: LogicalPlan): LogicalPlan 22 | ``` 23 | 24 | `apply`...FIXME 25 | 26 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 27 | -------------------------------------------------------------------------------- /docs/logical-optimizations/CombineUnions.md: -------------------------------------------------------------------------------- 1 | # CombineUnions Logical Optimization 2 | 3 | `CombineUnions` is a [base logical optimization](../catalyst/Optimizer.md#batches) that <>. 4 | 5 | `CombineUnions` is part of the [Union](../catalyst/Optimizer.md#Union) once-executed batch in the standard batches of the [Logical Optimizer](../catalyst/Optimizer.md). 6 | 7 | `CombineUnions` is simply a <> for transforming <>, i.e. `Rule[LogicalPlan]`. 8 | 9 | [source, scala] 10 | ---- 11 | // FIXME Demo 12 | ---- 13 | 14 | ## Executing Rule 15 | 16 | ```scala 17 | apply(plan: LogicalPlan): LogicalPlan 18 | ``` 19 | 20 | `apply`...FIXME 21 | 22 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 23 | -------------------------------------------------------------------------------- /docs/logical-optimizations/ConvertToLocalRelation.md: -------------------------------------------------------------------------------- 1 | # ConvertToLocalRelation Logical Optimization 2 | 3 | `ConvertToLocalRelation` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-optimizations/ExtractPythonUDFFromAggregate.md: -------------------------------------------------------------------------------- 1 | # ExtractPythonUDFFromAggregate Logical Optimization 2 | 3 | ## Executing Rule { #apply } 4 | 5 | ??? note "Rule" 6 | 7 | ```scala 8 | apply( 9 | plan: LogicalPlan): LogicalPlan 10 | ``` 11 | 12 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 13 | 14 | `apply`...FIXME 15 | -------------------------------------------------------------------------------- /docs/logical-optimizations/OptimizeMetadataOnlyQuery.md: -------------------------------------------------------------------------------- 1 | # OptimizeMetadataOnlyQuery Logical Optimization 2 | 3 | `OptimizeMetadataOnlyQuery` is...FIXME 4 | 5 | ## Executing Rule 6 | 7 | ```scala 8 | apply(plan: LogicalPlan): LogicalPlan 9 | ``` 10 | 11 | `apply`...FIXME 12 | 13 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 14 | -------------------------------------------------------------------------------- /docs/logical-optimizations/OptimizeSubqueries.md: -------------------------------------------------------------------------------- 1 | # OptimizeSubqueries Logical Optimization 2 | 3 | `OptimizeSubqueries` is a [base logical optimization](../catalyst/Optimizer.md#batches) that <>. 4 | 5 | `OptimizeSubqueries` is part of the [Subquery](../catalyst/Optimizer.md#Subquery) once-executed batch in the standard batches of the [Logical Optimizer](../catalyst/Optimizer.md). 6 | 7 | `OptimizeSubqueries` is simply a <> for transforming <>, i.e. `Rule[LogicalPlan]`. 8 | 9 | [source, scala] 10 | ---- 11 | // FIXME Demo 12 | ---- 13 | 14 | === [[apply]] Executing Rule -- `apply` Method 15 | 16 | [source, scala] 17 | ---- 18 | apply(plan: LogicalPlan): LogicalPlan 19 | ---- 20 | 21 | NOTE: `apply` is part of the <> to execute (apply) a rule on a <> (e.g. <>). 22 | 23 | `apply`...FIXME 24 | -------------------------------------------------------------------------------- /docs/logical-optimizations/PruneFilters.md: -------------------------------------------------------------------------------- 1 | # PruneFilters 2 | 3 | `PruneFilters` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-optimizations/PruneHiveTablePartitions.md: -------------------------------------------------------------------------------- 1 | # PruneHiveTablePartitions Logical Optimization 2 | 3 | `PruneHiveTablePartitions` is a logical optimization for partitioned Hive tables. 4 | 5 | ## Creating Instance 6 | 7 | `PruneHiveTablePartitions` takes the following to be created: 8 | 9 | * [SparkSession](../SparkSession.md) 10 | 11 | `PruneHiveTablePartitions` is created when: 12 | 13 | * `HiveSessionStateBuilder` is requested to [customEarlyScanPushDownRules](../hive/HiveSessionStateBuilder.md#customEarlyScanPushDownRules) 14 | 15 | ## updateTableMeta 16 | 17 | ```scala 18 | updateTableMeta( 19 | relation: HiveTableRelation, 20 | prunedPartitions: Seq[CatalogTablePartition], 21 | partitionKeyFilters: ExpressionSet): CatalogTable 22 | ``` 23 | 24 | `updateTableMeta`...FIXME 25 | 26 | --- 27 | 28 | `updateTableMeta` is used when: 29 | 30 | * `PruneHiveTablePartitions` is [executed](#apply) (for a partitioned [HiveTableRelation](../hive/HiveTableRelation.md) logical operator under a filter) 31 | -------------------------------------------------------------------------------- /docs/logical-optimizations/PushDownLeftSemiAntiJoin.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: PushDownLeftSemiAntiJoin 3 | --- 4 | 5 | # PushDownLeftSemiAntiJoin Logical Optimization 6 | 7 | `PushDownLeftSemiAntiJoin` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/logical-optimizations/PushDownPredicates.md: -------------------------------------------------------------------------------- 1 | # PushDownPredicates Logical Optimization 2 | 3 | `PushDownPredicates` is a logical optimization (of the [Logical Optimizer](../catalyst/Optimizer.md#defaultBatches) and the [SparkOptimizer](../SparkOptimizer.md#defaultBatches)). 4 | 5 | `PushDownPredicates` is a [Catalyst rule](../catalyst/Rule.md) for transforming [logical plans](../logical-operators/LogicalPlan.md) (`Rule[LogicalPlan]`). 6 | 7 | ## Creating Instance 8 | 9 | `PushDownPredicates` takes no arguments to be created (and is a Scala `object`). 10 | 11 | ## Executing Rule 12 | 13 | ```scala 14 | apply( 15 | plan: LogicalPlan): LogicalPlan 16 | ``` 17 | 18 | `apply` requests the given [LogicalPlan](../logical-operators/LogicalPlan.md) to [transformWithPruning](../catalyst/TreeNode.md#transformWithPruning) operators with [FILTER](../catalyst/TreePattern.md#FILTER) or [JOIN](../catalyst/TreePattern.md#JOIN) tree patterns. 19 | 20 | `apply`...FIXME (migrate [PushDownPredicate](PushDownPredicate.md) logical optimization) 21 | 22 | `apply` is part of the [Rule](../catalyst/Rule.md#apply) abstraction. 23 | -------------------------------------------------------------------------------- /docs/logical-optimizations/SchemaPruning.md: -------------------------------------------------------------------------------- 1 | # SchemaPruning Logical Optimization 2 | 3 | `SchemaPruning` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-optimizations/UpdateAttributeNullability.md: -------------------------------------------------------------------------------- 1 | # UpdateAttributeNullability Logical Optimization 2 | 3 | `UpdateAttributeNullability` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-optimizations/V2Writes.md: -------------------------------------------------------------------------------- 1 | # V2Writes Logical Optimization 2 | 3 | `V2Writes` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/logical-optimizations/index.md: -------------------------------------------------------------------------------- 1 | # Logical Optimizations 2 | -------------------------------------------------------------------------------- /docs/metadata-columns/.pages: -------------------------------------------------------------------------------- 1 | title: Metadata Columns 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/metadata-columns/FileSourceConstantMetadataAttribute.md: -------------------------------------------------------------------------------- 1 | # FileSourceConstantMetadataAttribute 2 | 3 | ## Creating Metadata AttributeReference { #apply } 4 | 5 | ```scala 6 | apply( 7 | name: String, 8 | dataType: DataType, 9 | nullable: Boolean = false): AttributeReference 10 | ``` 11 | 12 | `apply` creates an `AttributeReference` with the following metadata: 13 | 14 | Metadata Key | Value 15 | -------------|------ 16 | [__metadata_col](#METADATA_COL_ATTR_KEY) | `true` 17 | [__file_source_metadata_col](FileSourceMetadataAttribute.md#FILE_SOURCE_METADATA_COL_ATTR_KEY) | `true` 18 | [__file_source_constant_metadata_col](#FILE_SOURCE_CONSTANT_METADATA_COL_ATTR_KEY) | `true` 19 | 20 | --- 21 | 22 | `apply` is used when: 23 | 24 | * [FileSourceStrategy](../execution-planning-strategies/FileSourceStrategy.md) execution planning strategy is executed (to plan a [LogicalRelation](../logical-operators/LogicalRelation.md) over a [HadoopFsRelation](../files/HadoopFsRelation.md)) 25 | -------------------------------------------------------------------------------- /docs/metadata-columns/FileSourceGeneratedMetadataAttribute.md: -------------------------------------------------------------------------------- 1 | # FileSourceGeneratedMetadataAttribute 2 | 3 | ## Creating Metadata AttributeReference { #apply } 4 | 5 | ```scala 6 | apply( 7 | name: String, 8 | dataType: DataType, 9 | nullable: Boolean = false): AttributeReference 10 | ``` 11 | 12 | `apply` creates an `AttributeReference` with the following metadata: 13 | 14 | Metadata Key | Value 15 | -------------|------ 16 | [__metadata_col](#METADATA_COL_ATTR_KEY) | `true` 17 | [__file_source_metadata_col](FileSourceMetadataAttribute.md#FILE_SOURCE_METADATA_COL_ATTR_KEY) | `true` 18 | [__file_source_generated_metadata_col](#FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY) | `true` 19 | 20 | --- 21 | 22 | `apply` is used when: 23 | 24 | * [FileSourceStrategy](../execution-planning-strategies/FileSourceStrategy.md) execution planning strategy is executed (to plan a [LogicalRelation](../logical-operators/LogicalRelation.md) over a [HadoopFsRelation](../files/HadoopFsRelation.md)) 25 | -------------------------------------------------------------------------------- /docs/metadata-columns/MetadataColumnsHelper.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: MetadataColumnsHelper 3 | --- 4 | 5 | # MetadataColumnsHelper Implicit Class 6 | 7 | `MetadataColumnsHelper` is a Scala implicit class for [Array[MetadataColumn]](#metadata). 8 | 9 | ## Creating Instance 10 | 11 | `MetadataColumnsHelper` takes the following to be created: 12 | 13 | * [MetadataColumn](../connector/catalog/MetadataColumn.md)s 14 | 15 | ## asStruct 16 | 17 | ```scala 18 | asStruct: StructType 19 | ``` 20 | 21 | `asStruct` creates a [StructType](../types/StructType.md) for the [MetadataColumns](#metadata). 22 | 23 | `asStruct` is used when: 24 | 25 | * FIXME 26 | -------------------------------------------------------------------------------- /docs/new-and-noteworthy/catalog-plugin-api-and-multi-catalog-support.md: -------------------------------------------------------------------------------- 1 | # Catalog Plugin API and Multi-Catalog Support 2 | 3 | **New in 3.0.0** 4 | 5 | Main abstractions: 6 | 7 | * [CatalogManager](../connector/catalog/CatalogManager.md) 8 | * [CatalogPlugin](../connector/catalog/CatalogPlugin.md) 9 | * [USE NAMESPACE](../sql/AstBuilder.md#visitUse) SQL statement 10 | * [SHOW CURRENT NAMESPACE](../sql/AstBuilder.md#visitShowCurrentNamespace) SQL statement 11 | 12 | ## Demo 13 | 14 | * [Developing CatalogPlugin](../demo/developing-catalogplugin.md) 15 | 16 | ## Example 17 | 18 | ```sql 19 | SHOW NAMESPACES; 20 | 21 | SHOW CURRENT NAMESPACE; 22 | 23 | CREATE NAMESPACE IF NOT EXISTS my_ns; 24 | 25 | USE NAMESPACE my_ns; 26 | 27 | SHOW CURRENT NAMESPACE; 28 | ``` 29 | 30 | ## References 31 | 32 | ### Articles 33 | 34 | * [SPIP: Identifiers for multi-catalog support](https://issues.apache.org/jira/browse/SPARK-27066) 35 | * [SPIP: Catalog API for table metadata](https://issues.apache.org/jira/browse/SPARK-27067) 36 | 37 | ### Videos 38 | 39 | * [Improving Apache Spark’s Reliability with DataSourceV2](https://youtu.be/rH_iCMuBCII) by Ryan Blue, Netflix 40 | -------------------------------------------------------------------------------- /docs/new-and-noteworthy/intervals.md: -------------------------------------------------------------------------------- 1 | # ANSI Intervals 2 | 3 | Spark SQL supports interval type defined by the ANSI SQL standard using `AnsiIntervalType`: 4 | 5 | * `DayTimeIntervalType` for day-time intervals 6 | * `YearMonthIntervalType` for year-month intervals 7 | 8 | Intervals can be positive and negative. 9 | 10 | ## Parquet 11 | 12 | ANSI intervals are supported by [parquet data source](../parquet/ParquetFileFormat.md) as follows: 13 | 14 | * `DayTimeIntervalType` is the same as `LongType` (`INT64`) 15 | * `YearMonthIntervalType` is the same as `IntegerType` (`INT32`) 16 | 17 | ## Demo 18 | 19 | ```text 20 | select date'today' - date'2021-01-01' as diff 21 | ``` 22 | -------------------------------------------------------------------------------- /docs/new-and-noteworthy/observable-metrics.md: -------------------------------------------------------------------------------- 1 | # Observable Metrics 2 | 3 | **New in 3.0.0** 4 | 5 | **Observable Metrics** feature adds a new [Dataset.observe](../dataset/index.md#observe) operator (that simply creates a [CollectMetrics](../logical-operators/CollectMetrics.md) unary logical operator). 6 | 7 | Observable Metrics were introduced to Apache Spark 3.0.0 as [SPARK-29345](https://issues.apache.org/jira/browse/SPARK-29345). 8 | 9 | ## References 10 | 11 | ### Articles 12 | 13 | * [Introducing Apache Spark 3.0](https://databricks.com/blog/2020/06/18/introducing-apache-spark-3-0-now-available-in-databricks-runtime-7-0.html) 14 | 15 | ### Videos 16 | 17 | * [Deep Dive into the New Features of Apache Spark 3.0](https://youtu.be/l6SuXvhorDY) 18 | -------------------------------------------------------------------------------- /docs/new-and-noteworthy/statistics.md: -------------------------------------------------------------------------------- 1 | # Statistics 2 | 3 | [Statistics](../cost-based-optimization/Statistics.md) are supported for the following only: 4 | 5 | 1. Hive Metastore tables for which `ANALYZE TABLE COMPUTE STATISTICS noscan` has been executed 6 | 1. [File-based data source tables](../files/FileFormat.md) for which the statistics are computed directly on the files of data 7 | 8 | ## Broadcast Join 9 | 10 | Broadcast Join can be automatically selected by the Spark Planner based on the [Statistics](../cost-based-optimization/Statistics.md) and the [spark.sql.autoBroadcastJoinThreshold](../configuration-properties.md#spark.sql.autoBroadcastJoinThreshold) configuration property. 11 | 12 | 13 | -------------------------------------------------------------------------------- /docs/noop/NoopBatchWrite.md: -------------------------------------------------------------------------------- 1 | # NoopBatchWrite 2 | 3 | `NoopBatchWrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/noop/NoopDataSource.md: -------------------------------------------------------------------------------- 1 | # NoopDataSource 2 | 3 | `NoopDataSource` is a [SimpleTableProvider](../connector/SimpleTableProvider.md) of writable [NoopTable](#getTable)s. 4 | 5 | ## Short Name 6 | 7 | `NoopDataSource` is registered under **noop** alias. 8 | 9 | ## Creating Table 10 | 11 | ```scala 12 | getTable( 13 | options: CaseInsensitiveStringMap): Table 14 | ``` 15 | 16 | `getTable` simply creates a [NoopTable](NoopTable.md). 17 | 18 | `getTable` is part of the [SimpleTableProvider](../connector/SimpleTableProvider.md#getTable) abstraction. 19 | -------------------------------------------------------------------------------- /docs/noop/NoopStreamingWrite.md: -------------------------------------------------------------------------------- 1 | # NoopStreamingWrite 2 | 3 | `NoopStreamingWrite` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/noop/index.md: -------------------------------------------------------------------------------- 1 | # Noop Connector 2 | 3 | **Noop Connector** is available as [noop](NoopDataSource.md#shortName) format. 4 | -------------------------------------------------------------------------------- /docs/parameterized-queries/.pages: -------------------------------------------------------------------------------- 1 | title: Parameterized Queries 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/parquet/.pages: -------------------------------------------------------------------------------- 1 | title: Parquet 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/parquet/ParquetFilters.md: -------------------------------------------------------------------------------- 1 | # ParquetFilters 2 | 3 | `ParquetFilters` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/parquet/ParquetOptions.md: -------------------------------------------------------------------------------- 1 | # ParquetOptions 2 | 3 | `ParquetOptions` is a `FileSourceOptions`. 4 | 5 | ## Creating Instance 6 | 7 | `ParquetOptions` takes the following to be created: 8 | 9 | * Parameters 10 | * [SQLConf](../SQLConf.md) 11 | 12 | `ParquetOptions` is created when: 13 | 14 | * `HiveOptions` is requested to `getHiveWriteCompression` 15 | * `ParquetFileFormat` is requested to [prepareWrite](ParquetFileFormat.md#prepareWrite) and [buildReaderWithPartitionValues](ParquetFileFormat.md#buildReaderWithPartitionValues) 16 | * `ParquetUtils` is requested to [inferSchema](ParquetUtils.md#inferSchema) 17 | * `ParquetScan` is requested to [createReaderFactory](ParquetScan.md#createReaderFactory) 18 | * `ParquetWrite` is requested to [prepareWrite](ParquetWrite.md#prepareWrite) 19 | 20 | ## Options 21 | 22 | ### mergeSchema 23 | 24 | Controls merging schemas from all Parquet part-files 25 | 26 | Default: [spark.sql.parquet.mergeSchema](../configuration-properties.md#spark.sql.parquet.mergeSchema) 27 | 28 | Used when: 29 | 30 | * `ParquetUtils` is requested to [infer schema](ParquetUtils.md#inferSchema) 31 | -------------------------------------------------------------------------------- /docs/parquet/ParquetWriteSupport.md: -------------------------------------------------------------------------------- 1 | # ParquetWriteSupport 2 | 3 | `ParquetWriteSupport` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/parquet/SparkToParquetSchemaConverter.md: -------------------------------------------------------------------------------- 1 | # SparkToParquetSchemaConverter 2 | 3 | `SparkToParquetSchemaConverter` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/partition-file-metadata-caching/index.md: -------------------------------------------------------------------------------- 1 | # Partition File Metadata Caching 2 | 3 | **Partition File Metadata Caching** uses [FileStatusCache](../files/FileStatusCache.md) to speed up partition file listing. 4 | 5 | ## Configuration Properties 6 | 7 | * [spark.sql.hive.filesourcePartitionFileCacheSize](../configuration-properties.md#spark.sql.hive.filesourcePartitionFileCacheSize) 8 | * [spark.sql.hive.manageFilesourcePartitions](../configuration-properties.md#spark.sql.hive.manageFilesourcePartitions) 9 | * [spark.sql.metadataCacheTTLSeconds](../configuration-properties.md#spark.sql.metadataCacheTTLSeconds) 10 | -------------------------------------------------------------------------------- /docs/physical-operators/AllTuples.md: -------------------------------------------------------------------------------- 1 | # AllTuples 2 | 3 | `AllTuples` is a [Distribution](Distribution.md) that indicates to use one partition only. 4 | -------------------------------------------------------------------------------- /docs/physical-operators/AtomicTableWriteExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: AtomicTableWriteExec 3 | --- 4 | 5 | # AtomicTableWriteExec Physical Commands 6 | 7 | `AtomicTableWriteExec` is an extension of the [V2TableWriteExec](V2TableWriteExec.md) abstraction for [physical commands](#implementations) that [writeToStagedTable](#writeToStagedTable) and support V1 write path (`SupportsV1Write`). 8 | 9 | ## Implementations 10 | 11 | * `AtomicCreateTableAsSelectExec` 12 | * `AtomicReplaceTableAsSelectExec` 13 | 14 | ## writeToStagedTable 15 | 16 | ```scala 17 | writeToStagedTable( 18 | stagedTable: StagedTable, 19 | writeOptions: CaseInsensitiveStringMap, 20 | ident: Identifier): Seq[InternalRow] 21 | ``` 22 | 23 | `writeToStagedTable`...FIXME 24 | -------------------------------------------------------------------------------- /docs/physical-operators/BaseJoinExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: BaseJoinExec 3 | --- 4 | 5 | # BaseJoinExec Physical Operators 6 | 7 | `BaseJoinExec` is an [extension](#contract) of the `BinaryExecNode` abstraction for [join physical operators](#implementations). 8 | 9 | ## Contract 10 | 11 | ###  Join Condition 12 | 13 | ```scala 14 | condition: Option[Expression] 15 | ``` 16 | 17 | ###  Join Type 18 | 19 | ```scala 20 | joinType: JoinType 21 | ``` 22 | 23 | [JoinType](../joins.md#JoinType) 24 | 25 | ###  Left Keys 26 | 27 | ```scala 28 | leftKeys: Seq[Expression] 29 | ``` 30 | 31 | ###  Right Keys 32 | 33 | ```scala 34 | rightKeys: Seq[Expression] 35 | ``` 36 | 37 | ## Implementations 38 | 39 | * [BroadcastNestedLoopJoinExec](BroadcastNestedLoopJoinExec.md) 40 | * CartesianProductExec 41 | * [HashJoin](HashJoin.md) 42 | * [ShuffledJoin](ShuffledJoin.md) 43 | -------------------------------------------------------------------------------- /docs/physical-operators/CacheTableExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CacheTableExec 3 | --- 4 | 5 | # CacheTableExec Physical Operator 6 | 7 | `CacheTableExec` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/physical-operators/CreateNamespaceExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: CreateNamespaceExec 3 | --- 4 | 5 | # CreateNamespaceExec Physical Command Operator 6 | 7 | `CreateNamespaceExec` is a leaf [V2CommandExec](V2CommandExec.md) that represents [CreateNamespace](../logical-operators/CreateNamespace.md) logical operator at execution (for non-[session catalogs](../connector/catalog/CatalogV2Util.md#isSessionCatalog)). 8 | 9 | ## Executing Command { #run } 10 | 11 | ??? note "V2CommandExec" 12 | 13 | ```scala 14 | run(): Seq[InternalRow] 15 | ``` 16 | 17 | `run` is part of the [V2CommandExec](V2CommandExec.md#run) abstraction. 18 | 19 | `run` requests the given [SupportsNamespaces](../connector/catalog/SupportsNamespaces.md) catalog to [check if the multi-part namespace exists](../connector/catalog/SupportsNamespaces.md#namespaceExists). 20 | 21 | Unless the namespace exists, `run` requests the [SupportsNamespaces](../connector/catalog/SupportsNamespaces.md) catalog to [create it](../connector/catalog/SupportsNamespaces.md#createNamespace) (with the `owner` property being the current user). 22 | -------------------------------------------------------------------------------- /docs/physical-operators/DebugExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: DebugExec 3 | --- 4 | 5 | # DebugExec Unary Physical Operator 6 | 7 | `DebugExec` is a [unary physical operator](UnaryExecNode.md). 8 | -------------------------------------------------------------------------------- /docs/physical-operators/DescribeTableExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: DescribeTableExec 3 | --- 4 | 5 | # DescribeTableExec Physical Command 6 | 7 | `DescribeTableExec` is a [physical command](V2CommandExec.md) that represents [DescribeRelation](../logical-operators/DescribeRelation.md) logical command at execution time. 8 | 9 | ## Creating Instance 10 | 11 | `DescribeTableExec` takes the following to be created: 12 | 13 | * Output [Attribute](../expressions/Attribute.md)s 14 | * [Table](../connector/Table.md) 15 | * `isExtended` flag 16 | 17 | `DescribeTableExec` is created when [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (and plans a [DescribeRelation](../logical-operators/DescribeRelation.md) logical command). 18 | -------------------------------------------------------------------------------- /docs/physical-operators/ExternalRDDScanExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ExternalRDDScanExec 3 | --- 4 | 5 | # ExternalRDDScanExec Leaf Physical Operator 6 | 7 | `ExternalRDDScanExec` is a leaf physical operator. 8 | -------------------------------------------------------------------------------- /docs/physical-operators/JoinCodegenSupport.md: -------------------------------------------------------------------------------- 1 | # JoinCodegenSupport 2 | 3 | `JoinCodegenSupport` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/physical-operators/ObjectProducerExec.md: -------------------------------------------------------------------------------- 1 | # ObjectProducerExec 2 | 3 | `ObjectProducerExec` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/physical-operators/OrderPreservingUnaryExecNode.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: OrderPreservingUnaryExecNode 3 | --- 4 | 5 | # OrderPreservingUnaryExecNode Unary Physical Operators 6 | 7 | `OrderPreservingUnaryExecNode` is a marker extension of the [UnaryExecNode](UnaryExecNode.md) and [AliasAwareQueryOutputOrdering](AliasAwareQueryOutputOrdering.md) abstractions for [unary physical operators](#implementations). 8 | 9 | ??? warning "FIXME Why is OrderPreservingUnaryExecNode needed?" 10 | Review the commit log to find out. 11 | 12 | ## Implementations 13 | 14 | * [ProjectExec](ProjectExec.md) 15 | * [SortAggregateExec](SortAggregateExec.md) 16 | * `TakeOrderedAndProjectExec` 17 | -------------------------------------------------------------------------------- /docs/physical-operators/OverwriteByExpressionExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: OverwriteByExpressionExec 3 | --- 4 | 5 | # OverwriteByExpressionExec Physical Command 6 | 7 | `OverwriteByExpressionExec` is a [V2TableWriteExec](V2TableWriteExec.md) with [BatchWriteHelper](BatchWriteHelper.md). 8 | 9 | ## Creating Instance 10 | 11 | `OverwriteByExpressionExec` takes the following to be created: 12 | 13 | * [SupportsWrite](../connector/SupportsWrite.md) 14 | * Delete Filters (`Array[Filter]`) 15 | * Write Options 16 | * [Physical Query Plan](SparkPlan.md) 17 | 18 | `OverwriteByExpressionExec` is created when [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed for [OverwriteByExpression](../logical-operators/OverwriteByExpression.md) with [DataSourceV2Relation](../logical-operators/DataSourceV2Relation.md) over writable tables (tables with [SupportsWrite](../connector/SupportsWrite.md) except with [V1_BATCH_WRITE](../connector/TableCapability.md#V1_BATCH_WRITE) capability). 19 | -------------------------------------------------------------------------------- /docs/physical-operators/RangeExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RangeExec 3 | --- 4 | 5 | # RangeExec Leaf Physical Operator 6 | 7 | `RangeExec` is a [leaf physical operator](SparkPlan.md#LeafExecNode). 8 | -------------------------------------------------------------------------------- /docs/physical-operators/ReusedExchangeExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ReusedExchangeExec 3 | --- 4 | 5 | # ReusedExchangeExec Leaf Physical Operator 6 | 7 | `ReusedExchangeExec` is a leaf physical operator. 8 | -------------------------------------------------------------------------------- /docs/physical-operators/ReusedSubqueryExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ReusedSubqueryExec 3 | --- 4 | 5 | # ReusedSubqueryExec Physical Operator 6 | 7 | `ReusedSubqueryExec` is a [BaseSubqueryExec](BaseSubqueryExec.md) and a `LeafExecNode` with a [child BaseSubqueryExec](#child) physical operator. 8 | 9 | `ReusedSubqueryExec` is a wrapper and delegates all activity (as a physical operator) to the [child BaseSubqueryExec](#child). 10 | 11 | ## Creating Instance 12 | 13 | `ReusedSubqueryExec` takes the following to be created: 14 | 15 | * Child [BaseSubqueryExec](BaseSubqueryExec.md) physical operator 16 | 17 | `ReusedSubqueryExec` is created when: 18 | 19 | * [ReuseAdaptiveSubquery](../physical-optimizations/ReuseAdaptiveSubquery.md) physical optimization is executed 20 | * [ReuseExchangeAndSubquery](../physical-optimizations/ReuseExchangeAndSubquery.md) physical optimization is executed 21 | -------------------------------------------------------------------------------- /docs/physical-operators/RowToColumnarExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RowToColumnarExec 3 | --- 4 | 5 | # RowToColumnarExec Physical Operator 6 | 7 | `RowToColumnarExec` is a `RowToColumnarTransition` for [Columnar Execution](../columnar-execution/index.md). 8 | 9 | `RowToColumnarExec` is the opposite of [ColumnarToRowExec](ColumnarToRowExec.md) physical operator. 10 | 11 | ## Creating Instance 12 | 13 | `RowToColumnarExec` takes the following to be created: 14 | 15 | * Child [SparkPlan](SparkPlan.md) 16 | 17 | `RowToColumnarExec` is created when: 18 | 19 | * [ApplyColumnarRulesAndInsertTransitions](../physical-optimizations/ApplyColumnarRulesAndInsertTransitions.md) physical optimization is executed 20 | -------------------------------------------------------------------------------- /docs/physical-operators/ShowTablePropertiesExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ShowTablePropertiesExec 3 | --- 4 | 5 | # ShowTablePropertiesExec Physical Command 6 | 7 | `ShowTablePropertiesExec` is a [physical command](V2CommandExec.md) that represents [ShowTableProperties](../logical-operators/ShowTableProperties.md) logical command at execution time (for non-[ShowTablePropertiesCommand](../logical-operators/ShowTablePropertiesCommand.md) cases). 8 | 9 | ## Creating Instance 10 | 11 | `ShowTablePropertiesExec` takes the following to be created: 12 | 13 | * Output [Attribute](../expressions/Attribute.md)s 14 | * [Table](../connector/Table.md) 15 | * (optional) Property Key 16 | 17 | `ShowTablePropertiesExec` is created when: 18 | 19 | * [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (and plans a [ShowTableProperties](../logical-operators/ShowTableProperties.md) logical command) 20 | -------------------------------------------------------------------------------- /docs/physical-operators/TableWriteExecHelper.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: TableWriteExecHelper 3 | --- 4 | 5 | # TableWriteExecHelper Unary Physical Commands 6 | 7 | `TableWriteExecHelper` is an extension of the [V2TableWriteExec](V2TableWriteExec.md) and `SupportsV1Write` abstractions for [unary physical commands](#implementations) that can [write to a table](#writeToTable). 8 | 9 | ## Implementations 10 | 11 | * `AtomicCreateTableAsSelectExec` 12 | * `AtomicReplaceTableAsSelectExec` 13 | * [CreateTableAsSelectExec](CreateTableAsSelectExec.md) 14 | * `ReplaceTableAsSelectExec` 15 | 16 | ## writeToTable 17 | 18 | ```scala 19 | writeToTable( 20 | catalog: TableCatalog, 21 | table: Table, 22 | writeOptions: CaseInsensitiveStringMap, 23 | ident: Identifier): Seq[InternalRow] 24 | ``` 25 | 26 | `writeToTable`...FIXME 27 | 28 | `writeToTable` is used when: 29 | 30 | * `CreateTableAsSelectExec` is requested to [run](CreateTableAsSelectExec.md#run) 31 | * `AtomicCreateTableAsSelectExec` is requested to `run` 32 | * `ReplaceTableAsSelectExec` is requested to `run` 33 | * `AtomicReplaceTableAsSelectExec` is requested to `run` 34 | -------------------------------------------------------------------------------- /docs/physical-operators/UnspecifiedDistribution.md: -------------------------------------------------------------------------------- 1 | # UnspecifiedDistribution 2 | 3 | `UnspecifiedDistribution` is a [Distribution](Distribution.md). 4 | 5 | [[requiredNumPartitions]] 6 | `UnspecifiedDistribution` specifies `None` for the Distribution.md#requiredNumPartitions[required number of partitions]. 7 | 8 | !!! note 9 | `None` for the required number of partitions indicates to use any number of partitions (possibly [spark.sql.shuffle.partitions](../configuration-properties.md#spark.sql.shuffle.partitions) configuration property). 10 | -------------------------------------------------------------------------------- /docs/physical-operators/WriteDeltaExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: WriteDeltaExec 3 | --- 4 | 5 | # WriteDeltaExec Physical Operator 6 | 7 | `WriteDeltaExec` is a [V2ExistingTableWriteExec](V2ExistingTableWriteExec.md) physical operator that represents a [WriteDelta](../logical-operators/WriteDelta.md) logical operator at execution time. 8 | 9 | ## Creating Instance 10 | 11 | `WriteDeltaExec` takes the following to be created: 12 | 13 | * Query Physical Plan ([SparkPlan](SparkPlan.md)) 14 | * `refreshCache` function (`() => Unit`) 15 | * `WriteDeltaProjections` 16 | * `DeltaWrite` 17 | 18 | `WriteDeltaExec` is created when: 19 | 20 | * [DataSourceV2Strategy](../execution-planning-strategies/DataSourceV2Strategy.md) execution planning strategy is executed (to plan a [WriteDelta](../logical-operators/WriteDelta.md) logical operator) 21 | -------------------------------------------------------------------------------- /docs/physical-operators/WriteFilesExec.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: WriteFilesExec 3 | --- 4 | 5 | # WriteFilesExec Unary Physical Operator 6 | 7 | `WriteFilesExec` is a [UnaryExecNode](UnaryExecNode.md) physical operator that represents [WriteFiles](../logical-operators/WriteFiles.md) logical operator at execution time. 8 | 9 | ## Creating Instance 10 | 11 | `WriteFilesExec` takes the following to be created: 12 | 13 | * Child [SparkPlan](SparkPlan.md) 14 | * [FileFormat](../files/FileFormat.md) 15 | * Partition Columns ([Attribute](../expressions/Attribute.md)s) 16 | * [BucketSpec](../bucketing/BucketSpec.md) 17 | * Options 18 | * Static Partitions (`TablePartitionSpec`) 19 | 20 | `WriteFilesExec` is created when: 21 | 22 | * [BasicOperators](../execution-planning-strategies/BasicOperators.md) execution planning strategy is executed (to plan a `WriteFiles` logical operator) 23 | -------------------------------------------------------------------------------- /docs/physical-operators/index.md: -------------------------------------------------------------------------------- 1 | # Physical Operators 2 | 3 | **Physical Operators** (_Physical Relational Operators_) are building blocks of physical query plans. 4 | 5 | **Physical Query Plan** is a tree of [nodes](../catalyst/TreeNode.md) of physical operators that in turn can have (trees of) [Catalyst expressions](../expressions/Expression.md). In other words, there are _at least_ two trees at every level (operator). 6 | 7 | The main abstraction is [SparkPlan](SparkPlan.md) that is a recursive data structure with zero, one, two or more child logical operators: 8 | 9 | * `LeafExecNode` 10 | * [UnaryExecNode](UnaryExecNode.md) 11 | * `BinaryExecNode` 12 | -------------------------------------------------------------------------------- /docs/physical-optimizations/.pages: -------------------------------------------------------------------------------- 1 | title: Physical Optimizations 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/physical-optimizations/AdjustShuffleExchangePosition.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: AdjustShuffleExchangePosition 3 | --- 4 | 5 | # AdjustShuffleExchangePosition Physical Optimization 6 | 7 | `AdjustShuffleExchangePosition` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/physical-optimizations/PlanAdaptiveSubqueries.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: PlanAdaptiveSubqueries 3 | --- 4 | 5 | # PlanAdaptiveSubqueries Physical Optimization 6 | 7 | `PlanAdaptiveSubqueries` is a physical query plan optimization in [Adaptive Query Execution](../adaptive-query-execution/index.md). 8 | 9 | `PlanAdaptiveSubqueries` is a [Catalyst Rule](../catalyst/Rule.md) for transforming [physical plans](../physical-operators/SparkPlan.md) (`Rule[SparkPlan]`). 10 | -------------------------------------------------------------------------------- /docs/physical-optimizations/RemoveRedundantProjects.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RemoveRedundantProjects 3 | --- 4 | 5 | # RemoveRedundantProjects Physical Optimization 6 | 7 | `RemoveRedundantProjects` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/physical-optimizations/RemoveRedundantSorts.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: RemoveRedundantSorts 3 | --- 4 | 5 | # RemoveRedundantSorts Physical Optimization 6 | 7 | `RemoveRedundantSorts` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/physical-optimizations/ReuseSubquery.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ReuseSubquery 3 | --- 4 | 5 | # ReuseSubquery Physical Optimization 6 | 7 | `ReuseSubquery` is a *physical query optimization* (aka _physical query preparation rule_ or simply _preparation rule_) that `QueryExecution` [uses](../QueryExecution.md#preparations) to optimize the physical plan of a structured query. 8 | 9 | Technically, `ReuseSubquery` is just a catalyst/Rule.md[Catalyst rule] for transforming SparkPlan.md[physical query plans], i.e. `Rule[SparkPlan]`. 10 | 11 | `ReuseSubquery` is part of [preparations](../QueryExecution.md#preparations) batch of physical query plan rules and is executed when `QueryExecution` is requested for the [optimized physical query plan](../QueryExecution.md#executedPlan) (i.e. in *executedPlan* phase of a query execution). 12 | 13 | === [[apply]] `apply` Method 14 | 15 | [source, scala] 16 | ---- 17 | apply(plan: SparkPlan): SparkPlan 18 | ---- 19 | 20 | NOTE: `apply` is part of catalyst/Rule.md#apply[Rule Contract] to apply a rule to a SparkPlan.md[physical plan]. 21 | 22 | `apply`...FIXME 23 | -------------------------------------------------------------------------------- /docs/physical-optimizations/ValidateSparkPlan.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: ValidateSparkPlan 3 | --- 4 | 5 | # ValidateSparkPlan Physical Optimization 6 | 7 | `ValidateSparkPlan` is...FIXME 8 | -------------------------------------------------------------------------------- /docs/physical-optimizations/index.md: -------------------------------------------------------------------------------- 1 | # Physical Optimizations 2 | 3 | **Physical Optimizations** (_physical query preparation rules_, _preparation rules_) are used by [QueryExecution](../QueryExecution.md#preparations) to optimize (_transform_) the physical plan of structured queries. 4 | 5 | `QueryExecution` uses physical optimizations when requested for the [optimized physical query plan](../QueryExecution.md#executedPlan) (i.e. in _executedPlan_ phase of a query execution). 6 | 7 | !!! note "FIXME Mention AQE" 8 | -------------------------------------------------------------------------------- /docs/query-execution/index.md: -------------------------------------------------------------------------------- 1 | # Query Execution 2 | 3 | [QueryExecution](../QueryExecution.md) is the query execution engine of Spark SQL. 4 | -------------------------------------------------------------------------------- /docs/runtime-filtering/.pages: -------------------------------------------------------------------------------- 1 | title: Runtime Filtering 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/runtime-filtering/index.md: -------------------------------------------------------------------------------- 1 | # Runtime Filtering 2 | 3 | **Runtime Filtering** is an optimization of join queries by pre-filtering one side of a join using [Bloom Filter](../bloom-filter-join/index.md) or [InSubquery](../expressions/InSubquery.md) predicate based on the values from the other side of the join. 4 | 5 | Runtime Filtering uses [InjectRuntimeFilter](../logical-optimizations/InjectRuntimeFilter.md) logical optimization to inject either [Bloom Filter](../bloom-filter-join/index.md) or [InSubquery](../expressions/InSubquery.md) predicate based on [spark.sql.optimizer.runtime.bloomFilter.enabled](../configuration-properties.md#spark.sql.optimizer.runtime.bloomFilter.enabled) configuration property. 6 | -------------------------------------------------------------------------------- /docs/spark-connect.md: -------------------------------------------------------------------------------- 1 | # Spark Connect 2 | 3 | **Spark Connect** has moved to its own [The Internals of Spark Connect](https://books.japila.pl/spark-connect-internals) online book. 4 | -------------------------------------------------------------------------------- /docs/spark-logging.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc # Hide table of contents 4 | --- 5 | 6 | # Logging 7 | 8 | Apache Spark uses [Apache Log4j 2](https://logging.apache.org/log4j/2.x/index.html) for logging. 9 | 10 | !!! note 11 | Learn more in [The Internals of Apache Spark]({{ book.spark_core }}/spark-logging/) online book. 12 | -------------------------------------------------------------------------------- /docs/spark-sql-dataset-rdd.md: -------------------------------------------------------------------------------- 1 | # Datasets, DataFrames and RDDs 2 | 3 | Many may have been asking yourself why they should be using Datasets rather than the foundation of all Spark - RDDs using case classes. 4 | 5 | This document collects advantages of `Dataset` vs `RDD[CaseClass]` to answer [the question Dan has asked on twitter](https://twitter.com/danosipov/status/704421546203308033): 6 | 7 | > "In #Spark, what is the advantage of a DataSet over an RDD[CaseClass]?" 8 | 9 | ## Saving to or Writing from Data Sources 10 | 11 | With Dataset API, loading data from a data source or saving it to one is as simple as using [SparkSession.read](SparkSession.md#read) or [Dataset.write](dataset/index.md#write) methods, appropriately. 12 | 13 | ## Accessing Fields / Columns 14 | 15 | You `select` columns in a datasets without worrying about the positions of the columns. 16 | 17 | In RDD, you have to do an additional hop over a case class and access fields by name. 18 | -------------------------------------------------------------------------------- /docs/spark-sql-performance-tuning.md: -------------------------------------------------------------------------------- 1 | # Spark SQL's Performance Tuning Tips and Tricks (aka Case Studies) 2 | 3 | From time to time I'm lucky enough to find ways to optimize structured queries in Spark SQL. These findings (or discoveries) usually fall into a study category than a single topic and so the goal of *Spark SQL's Performance Tuning Tips and Tricks* chapter is to have a single place for the so-called tips and tricks. 4 | 5 | . spark-sql-performance-tuning-groupBy-aggregation.md[Number of Partitions for groupBy Aggegration] 6 | 7 | ## Others 8 | 9 | . Avoid `ObjectType` [it turns whole-stage Java code generation off](physical-optimizations/CollapseCodegenStages.md#insertWholeStageCodegen-ObjectType). 10 | 11 | . Keep [whole-stage codegen requirements](physical-optimizations/CollapseCodegenStages.md#supportCodegen) in mind, in particular avoid physical operators with [supportCodegen](physical-operators/CodegenSupport.md#supportCodegen) flag off. 12 | -------------------------------------------------------------------------------- /docs/standard-functions/.pages: -------------------------------------------------------------------------------- 1 | title: Standard Functions 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/storage-partitioned-joins/.pages: -------------------------------------------------------------------------------- 1 | title: Storage-Partitioned Joins 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/subexpression-elimination/.pages: -------------------------------------------------------------------------------- 1 | title: Subexpression Elimination 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/subqueries/.pages: -------------------------------------------------------------------------------- 1 | title: Subqueries 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/table-valued-functions/.pages: -------------------------------------------------------------------------------- 1 | title: Table-Valued Functions 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/thrift-server/.pages: -------------------------------------------------------------------------------- 1 | title: Spark Thrift Server 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/thrift-server/SparkSQLEnv.md: -------------------------------------------------------------------------------- 1 | # SparkSQLEnv 2 | 3 | CAUTION: FIXME 4 | -------------------------------------------------------------------------------- /docs/time-travel/.pages: -------------------------------------------------------------------------------- 1 | title: Time Travel 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/time-travel/TimeTravelSpec.md: -------------------------------------------------------------------------------- 1 | # TimeTravelSpec 2 | 3 | `TimeTravelSpec` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/transactional-writes/.pages: -------------------------------------------------------------------------------- 1 | title: Transactional Writes 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/transactional-writes/index.md: -------------------------------------------------------------------------------- 1 | # Transactional Writes in File-Based Connectors 2 | 3 | [File-based connectors](../files/index.md) use [spark.sql.sources.commitProtocolClass](../configuration-properties.md#spark.sql.sources.commitProtocolClass) configuration property for the class that is responsible for transactional writes. 4 | 5 | ## Logging 6 | 7 | Enable the following loggers: 8 | 9 | * [FileFormatWriter](../files/FileFormatWriter.md#logging) 10 | * [ParquetUtils](../parquet/ParquetUtils.md#logging) 11 | * [SQLHadoopMapReduceCommitProtocol](SQLHadoopMapReduceCommitProtocol.md#logging) 12 | 13 | ## Learn More 14 | 15 | 1. [Transactional Writes to Cloud Storage on Databricks](https://www.databricks.com/blog/2017/05/31/transactional-writes-cloud-storage.html) 16 | -------------------------------------------------------------------------------- /docs/tungsten/UnsafeRowSerializerInstance.md: -------------------------------------------------------------------------------- 1 | # UnsafeRowSerializerInstance 2 | 3 | `UnsafeRowSerializerInstance` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/types/.pages: -------------------------------------------------------------------------------- 1 | title: Data Types 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/types/AbstractDataType.md: -------------------------------------------------------------------------------- 1 | # AbstractDataType 2 | 3 | `AbstractDataType` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/types/CalendarInterval.md: -------------------------------------------------------------------------------- 1 | # CalendarInterval 2 | 3 | `CalendarInterval` represents a calendar interval. 4 | 5 | ## Creating Instance 6 | 7 | `CalendarInterval` takes the following to be created: 8 | 9 | * Months 10 | * Days 11 | * Microseconds 12 | 13 | `CalendarInterval` is created when: 14 | 15 | * `CALENDAR_INTERVAL` utility is used to `extract` a `CalendarInterval` from a `ByteBuffer` 16 | * `ColumnVector` is requested to [getInterval](../vectorized-decoding/ColumnVector.md#getInterval) 17 | * `IntervalUtils` utilities are used 18 | * `DateTimeUtils` utility is used to `subtractDates` 19 | * `UnsafeRow` is requested to [getInterval](../UnsafeRow.md#getInterval) 20 | * `UnsafeArrayData` is requested to `getInterval` 21 | * `Literal` utility is used to [create the default value for CalendarIntervalType](../expressions/Literal.md#default) 22 | * `TemporalSequenceImpl` is requested for the `defaultStep` 23 | 24 | ## Examples 25 | 26 | ```text 27 | 0 seconds 28 | 5 years 29 | 2 months 30 | 10 days 31 | 2 hours 32 | 1 minute 33 | ``` 34 | -------------------------------------------------------------------------------- /docs/types/DataType.md: -------------------------------------------------------------------------------- 1 | # DataType 2 | 3 | `DataType` is an [extension](#contract) of the [AbstractDataType](AbstractDataType.md) abstraction for [data types](#implementations) in Spark SQL. 4 | 5 | ## Contract 6 | 7 | ### asNullable 8 | 9 | ```scala 10 | asNullable: DataType 11 | ``` 12 | 13 | ### Default Size 14 | 15 | ```scala 16 | defaultSize: Int 17 | ``` 18 | 19 | Default size of a value of this data type 20 | 21 | Used when: 22 | 23 | * [ResolveGroupingAnalytics](../logical-analysis-rules/ResolveGroupingAnalytics.md) logical resolution is executed 24 | * `CommandUtils` is used to [statExprs](../CommandUtils.md#statExprs) 25 | * `JoinEstimation` is used to [estimateInnerOuterJoin](../cost-based-optimization/JoinEstimation.md#estimateInnerOuterJoin) 26 | * _others_ 27 | 28 | ## Implementations 29 | 30 | * [ArrayType](ArrayType.md) 31 | * [AtomicType](AtomicType.md) 32 | * [StructType](StructType.md) 33 | * _others_ 34 | -------------------------------------------------------------------------------- /docs/types/Metadata.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/docs/types/Metadata.md -------------------------------------------------------------------------------- /docs/ui/.pages: -------------------------------------------------------------------------------- 1 | title: Web UI 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/ui/AllExecutionsPage.md: -------------------------------------------------------------------------------- 1 | # AllExecutionsPage 2 | 3 | `AllExecutionsPage` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ui/ExecutionPage.md: -------------------------------------------------------------------------------- 1 | # ExecutionPage 2 | 3 | `ExecutionPage` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ui/SQLAppStatusListener.md: -------------------------------------------------------------------------------- 1 | # SQLAppStatusListener 2 | 3 | `SQLAppStatusListener` is a `SparkListener` ([Spark Core]({{ book.spark_core }}/SparkListener/)). 4 | 5 | ## Creating Instance 6 | 7 | `SQLAppStatusListener` takes the following to be created: 8 | 9 | * `SparkConf` ([Spark Core]({{ book.spark_core }}/SparkConf)) 10 | * `ElementTrackingStore` ([Spark Core]({{ book.spark_core }}/status/ElementTrackingStore)) 11 | * `live` flag 12 | 13 | `SQLAppStatusListener` is created when: 14 | 15 | * `SharedState` is created (and initializes a [SQLAppStatusStore](../SharedState.md#statusStore)) 16 | * `SQLHistoryServerPlugin` is requested to create `SparkListener`s 17 | -------------------------------------------------------------------------------- /docs/ui/SQLAppStatusStore.md: -------------------------------------------------------------------------------- 1 | # SQLAppStatusStore 2 | 3 | `SQLAppStatusStore` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ui/index.md: -------------------------------------------------------------------------------- 1 | # SQL / DataFrame UI 2 | 3 | Structured queries can be monitored using [web UI](SQLTab.md) that attaches the following two pages: 4 | 5 | * [AllExecutionsPage](AllExecutionsPage.md) 6 | * [ExecutionPage](ExecutionPage.md) 7 | -------------------------------------------------------------------------------- /docs/user-defined-functions/.pages: -------------------------------------------------------------------------------- 1 | title: User-Defined Functions 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/user-defined-functions/index.md: -------------------------------------------------------------------------------- 1 | # User-Defined Functions 2 | -------------------------------------------------------------------------------- /docs/variable-substitution.md: -------------------------------------------------------------------------------- 1 | # Variable Substitution 2 | 3 | Spark SQL (and Spark Thrift Server) supports [Variable Substitution](sql/VariableSubstitution.md) in SQL commands using syntax like `${var}`, `${system:var}`, and `${env:var}`. 4 | 5 | !!! note 6 | `VariableSubstitution` is meant for SQL commands mainly (if not exclusively) since in programming languages there are other means to achieve it (e.g., [String Interpolation](https://docs.scala-lang.org/overviews/core/string-interpolation.html) in Scala). 7 | 8 | ## spark.sql.variable.substitute 9 | 10 | [spark.sql.variable.substitute](configuration-properties.md#spark.sql.variable.substitute) configuration property is used to enable variable substitution. 11 | -------------------------------------------------------------------------------- /docs/vectorized-decoding/ColumnVector.md: -------------------------------------------------------------------------------- 1 | # ColumnVector 2 | 3 | `ColumnVector` is an [abstraction](#contract) of [in-memory columnar data](#implementations) (_vectors_) with elements of a given [DataType](#type). 4 | 5 | `ColumnVector` is expected to be reused during the entire data loading process, to avoid allocating memory again and again. 6 | 7 | `ColumnVector` is meant to maximize CPU efficiency but not to minimize storage footprint. Implementations should prefer computing efficiency over storage efficiency when design the format. Since it is expected to reuse the ColumnVector instance while loading data, the storage footprint is negligible. 8 | 9 | ## Implementations 10 | 11 | * `ArrowColumnVector` 12 | * `ConstantColumnVector` 13 | * `OrcColumnVector` 14 | * [WritableColumnVector](WritableColumnVector.md) 15 | 16 | ## Creating Instance 17 | 18 | `ColumnVector` takes the following to be created: 19 | 20 | * [DataType](../types/DataType.md) 21 | 22 | !!! note "Abstract Class" 23 | `ColumnVector` is an abstract class and cannot be created directly. It is created indirectly for the [concrete ColumnVectors](#implementations). 24 | -------------------------------------------------------------------------------- /docs/vectorized-decoding/OffHeapColumnVector.md: -------------------------------------------------------------------------------- 1 | # OffHeapColumnVector 2 | 3 | `OffHeapColumnVector` is a concrete [WritableColumnVector](WritableColumnVector.md). 4 | -------------------------------------------------------------------------------- /docs/vectorized-decoding/index.md: -------------------------------------------------------------------------------- 1 | # Vectorized Parquet Decoding (Reader) 2 | 3 | **Vectorized Parquet Decoding** (**Vectorized Parquet Reader**) allows for reading datasets in parquet format in batches, i.e. rows are decoded in batches. That aims at improving memory locality and cache utilization. 4 | 5 | Quoting [SPARK-12854 Vectorize Parquet reader](https://issues.apache.org/jira/browse/SPARK-12854): 6 | 7 | > The parquet encodings are largely designed to decode faster in batches, column by column. This can speed up the decoding considerably. 8 | 9 | Vectorized Parquet Decoding is used exclusively when `ParquetFileFormat` is requested for a [data reader](../parquet/ParquetFileFormat.md#buildReaderWithPartitionValues) when [spark.sql.parquet.enableVectorizedReader](../configuration-properties.md#spark.sql.parquet.enableVectorizedReader) property is enabled (`true`) and the read schema uses [AtomicTypes](../types/AtomicType.md) data types only. 10 | 11 | Vectorized Parquet Decoding uses [VectorizedParquetRecordReader](../parquet/VectorizedParquetRecordReader.md) for vectorized decoding (and [ParquetReadSupport](../parquet/ParquetReadSupport.md) otherwise). 12 | -------------------------------------------------------------------------------- /docs/vectorized-query-execution/index.md: -------------------------------------------------------------------------------- 1 | # Vectorized Query Execution 2 | 3 | **Vectorized Query Execution** is...FIXME 4 | 5 | Vectorized Query Execution starts with [Columnar Scan](../vectorized-decoding/index.md). 6 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/.pages: -------------------------------------------------------------------------------- 1 | title: Whole-Stage Code Generation 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/Block.md: -------------------------------------------------------------------------------- 1 | # Block 2 | 3 | `Block` is an extension of the [TreeNode](../catalyst/TreeNode.md) abstraction for [nodes](#implementations) that represent a block of Java code. 4 | 5 | ## Implementations 6 | 7 | ### CodeBlock 8 | 9 | ### EmptyBlock 10 | 11 | ## Textual Representation 12 | 13 | ```scala 14 | toString: String 15 | ``` 16 | 17 | `toString`...FIXME 18 | 19 | `toString` is part of Java's `java.lang.Object` abstraction. 20 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/GenerateColumnAccessor.md: -------------------------------------------------------------------------------- 1 | # GenerateColumnAccessor 2 | 3 | `GenerateColumnAccessor` is a [CodeGenerator](CodeGenerator.md). 4 | 5 | ## Creating ColumnarIterator 6 | 7 | ```scala 8 | create( 9 | columnTypes: Seq[DataType]): ColumnarIterator 10 | ``` 11 | 12 | `create` is part of the [CodeGenerator](CodeGenerator.md#create) abstraction. 13 | 14 | `create`...FIXME 15 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/GenerateMutableProjection.md: -------------------------------------------------------------------------------- 1 | # GenerateMutableProjection 2 | 3 | === [[create]] Creating MutableProjection -- `create` Internal Method 4 | 5 | [source, scala] 6 | ---- 7 | create( 8 | expressions: Seq[Expression], 9 | useSubexprElimination: Boolean): MutableProjection 10 | ---- 11 | 12 | `create`...FIXME 13 | 14 | NOTE: `create` is used when...FIXME 15 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/GenerateOrdering.md: -------------------------------------------------------------------------------- 1 | # GenerateOrdering 2 | 3 | === [[create]] Creating BaseOrdering -- `create` Method 4 | 5 | [source, scala] 6 | ---- 7 | create(ordering: Seq[SortOrder]): BaseOrdering 8 | create(schema: StructType): BaseOrdering 9 | ---- 10 | 11 | `create` is part of the [CodeGenerator](CodeGenerator.md#create) abstraction. 12 | 13 | `create`...FIXME 14 | 15 | === [[genComparisons]] `genComparisons` Method 16 | 17 | [source, scala] 18 | ---- 19 | genComparisons(ctx: CodegenContext, schema: StructType): String 20 | ---- 21 | 22 | `genComparisons`...FIXME 23 | 24 | NOTE: `genComparisons` is used when...FIXME 25 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/GeneratePredicate.md: -------------------------------------------------------------------------------- 1 | # GeneratePredicate 2 | 3 | === [[create]] Creating Predicate -- `create` Method 4 | 5 | [source, scala] 6 | ---- 7 | create(predicate: Expression): Predicate 8 | ---- 9 | 10 | `create` is part of the [CodeGenerator](CodeGenerator.md#create) abstraction. 11 | 12 | `create`...FIXME 13 | -------------------------------------------------------------------------------- /docs/whole-stage-code-generation/GenerateSafeProjection.md: -------------------------------------------------------------------------------- 1 | # GenerateSafeProjection 2 | 3 | `GenerateSafeProjection` utility is a [CodeGenerator](CodeGenerator.md). 4 | 5 | ```scala 6 | CodeGenerator[Seq[Expression], Projection] 7 | ``` 8 | 9 | `GenerateSafeProjection` is used when: 10 | 11 | * `SafeProjection` utility is used to `createCodeGeneratedObject` 12 | * `DeserializeToObjectExec` physical operator is [executed](../physical-operators/DeserializeToObjectExec.md#doExecute) 13 | * `ObjectOperator` utility is used to `deserializeRowToObject` 14 | * `ComplexTypedAggregateExpression` is requested for `inputRowToObj` and `bufferRowToObject` 15 | 16 | ## Creating Projection 17 | 18 | ```scala 19 | create( 20 | expressions: Seq[Expression]): Projection 21 | ``` 22 | 23 | `create`...FIXME 24 | 25 | `create` is part of the [CodeGenerator](CodeGenerator.md#create) abstraction. 26 | -------------------------------------------------------------------------------- /docs/window-functions/.pages: -------------------------------------------------------------------------------- 1 | title: Window Functions 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/window-functions/Window.md: -------------------------------------------------------------------------------- 1 | # Window 2 | 3 | `Window` utility allows to define a [WindowSpec](WindowSpec.md). 4 | 5 | ## Frame Boundaries 6 | 7 | Numeric Value | Frame Boundary 8 | -------|--------------- 9 | `Long.MinValue` | [Window.unboundedPreceding](Window.md#unboundedPreceding) 10 | `0` | [Window.currentRow](Window.md#currentRow) 11 | `Long.MaxValue` | [Window.unboundedFollowing](Window.md#unboundedFollowing) 12 | -------------------------------------------------------------------------------- /graffles/AQEOptimizer.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/AQEOptimizer.graffle -------------------------------------------------------------------------------- /graffles/Dataset.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/Dataset.graffle -------------------------------------------------------------------------------- /graffles/QueryExecution-execution-pipeline.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/QueryExecution-execution-pipeline.graffle -------------------------------------------------------------------------------- /graffles/ReorderJoin-createOrderedJoin-four-plans-after.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/ReorderJoin-createOrderedJoin-four-plans-after.graffle -------------------------------------------------------------------------------- /graffles/ReorderJoin-createOrderedJoin-four-plans-before.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/ReorderJoin-createOrderedJoin-four-plans-before.graffle -------------------------------------------------------------------------------- /graffles/ReorderJoin-createOrderedJoin-four-plans.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/ReorderJoin-createOrderedJoin-four-plans.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-Analyzer.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-Analyzer.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-CatalogImpl-createExternalTable.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-CatalogImpl-createExternalTable.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-CatalogImpl.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-CatalogImpl.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-GenerateExec-doExecute.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-GenerateExec-doExecute.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-GlobalTempViewManager-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-GlobalTempViewManager-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-GlobalTempViewManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-GlobalTempViewManager.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-HiveExternalCatalog.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-HiveExternalCatalog.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-HiveMetastoreCatalog.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-HiveMetastoreCatalog.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-HiveSessionCatalog.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-HiveSessionCatalog.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-HiveSessionStateBuilder-SessionState.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-HiveSessionStateBuilder-SessionState.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-HiveSessionStateBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-HiveSessionStateBuilder.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-InputAdapter-doProduce.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-InputAdapter-doProduce.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-RuntimeConfig.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-RuntimeConfig.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-ScalaUDAF-initialize.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-ScalaUDAF-initialize.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-ScalaUDAF-merge.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-ScalaUDAF-merge.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-ScalaUDAF-update.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-ScalaUDAF-update.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SessionCatalog.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SessionCatalog.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SessionState.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SessionState.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SparkOptimizer.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SparkOptimizer.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SparkPlan-execute-pipeline.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SparkPlan-execute-pipeline.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SparkPlan-execute.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SparkPlan-execute.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SparkPlan-executeTake.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SparkPlan-executeTake.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-SparkSqlAstBuilder.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-SparkSqlAstBuilder.graffle -------------------------------------------------------------------------------- /graffles/spark-sql-UserDefinedAggregateFunction.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/spark-sql-internals/112acd211dd2609baa880f6418eae24b50dbbc76/graffles/spark-sql-UserDefinedAggregateFunction.graffle -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | git+https://${GH_TOKEN}@github.com/squidfunk/mkdocs-material-insiders.git 2 | mkdocs-minify-plugin>=0.7.2 3 | mkdocs-git-revision-date-localized-plugin>=1.2.2 4 | mkdocs-git-revision-date-plugin>=0.3.2 5 | mkdocs-awesome-pages-plugin>=2.9.2 6 | mkdocs-redirects>=1.2.1 7 | mkdocs-macros-plugin>=1.0.5 8 | --------------------------------------------------------------------------------