class SparkSessionExtensions extends AnyRef
:: Experimental :: Holder for injection points to the SparkSession. We make NO guarantee about the stability regarding binary compatibility and source compatibility of methods here.
This current provides the following extension points:
- Analyzer Rules.
 - Check Analysis Rules.
 - Cache Plan Normalization Rules.
 - Optimizer Rules.
 - Pre CBO Rules.
 - Planning Strategies.
 - Customized Parser.
 - (External) Catalog listeners.
 - Columnar Rules.
 - Adaptive Query Stage Preparation Rules.
 - Adaptive Query Execution Runtime Optimizer Rules.
 - Adaptive Query Stage Optimizer Rules.
 
The extensions can be used by calling withExtensions on the SparkSession.Builder, for
example:
SparkSession.builder() .master("...") .config("...", true) .withExtensions { extensions => extensions.injectResolutionRule { session => ... } extensions.injectParser { (session, parser) => ... } } .getOrCreate()
The extensions can also be used by setting the Spark SQL configuration property
spark.sql.extensions. Multiple extensions can be set using a comma-separated list. For example:
SparkSession.builder() .master("...") .config("spark.sql.extensions", "org.example.MyExtensions,org.example.YourExtensions") .getOrCreate() class MyExtensions extends Function1[SparkSessionExtensions, Unit] { override def apply(extensions: SparkSessionExtensions): Unit = { extensions.injectResolutionRule { session => ... } extensions.injectParser { (session, parser) => ... } } } class YourExtensions extends SparkSessionExtensionsProvider { override def apply(extensions: SparkSessionExtensions): Unit = { extensions.injectResolutionRule { session => ... } extensions.injectFunction(...) } }
Note that none of the injected builders should assume that the SparkSession is fully initialized and should not touch the session's internals (e.g. the SessionState).
- Annotations
 - @DeveloperApi() @Experimental() @Unstable()
 - Source
 - SparkSessionExtensions.scala
 
- Alphabetic
 - By Inheritance
 
- SparkSessionExtensions
 - AnyRef
 - Any
 
- Hide All
 - Show All
 
- Public
 - All
 
Instance Constructors
-  new SparkSessionExtensions()
 
Type Members
-  type CheckRuleBuilder = (SparkSession) ⇒ (LogicalPlan) ⇒ Unit
 -  type ColumnarRuleBuilder = (SparkSession) ⇒ ColumnarRule
 -  type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder)
 -  type ParserBuilder = (SparkSession, ParserInterface) ⇒ ParserInterface
 -  type QueryStageOptimizerRuleBuilder = (SparkSession) ⇒ Rule[SparkPlan]
 -  type QueryStagePrepRuleBuilder = (SparkSession) ⇒ Rule[SparkPlan]
 -  type RuleBuilder = (SparkSession) ⇒ Rule[LogicalPlan]
 -  type StrategyBuilder = (SparkSession) ⇒ Strategy
 -  type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder)
 
Value Members
- 
      
      
      
        
      
    
      
        final 
        def
      
      
        !=(arg0: Any): Boolean
      
      
      
- Definition Classes
 - AnyRef → Any
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        ##(): Int
      
      
      
- Definition Classes
 - AnyRef → Any
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        ==(arg0: Any): Boolean
      
      
      
- Definition Classes
 - AnyRef → Any
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        asInstanceOf[T0]: T0
      
      
      
- Definition Classes
 - Any
 
 -  def buildPlanNormalizationRules(session: SparkSession): Seq[Rule[LogicalPlan]]
 - 
      
      
      
        
      
    
      
        
        def
      
      
        clone(): AnyRef
      
      
      
- Attributes
 - protected[lang]
 - Definition Classes
 - AnyRef
 - Annotations
 - @throws( ... ) @native()
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        eq(arg0: AnyRef): Boolean
      
      
      
- Definition Classes
 - AnyRef
 
 - 
      
      
      
        
      
    
      
        
        def
      
      
        equals(arg0: Any): Boolean
      
      
      
- Definition Classes
 - AnyRef → Any
 
 - 
      
      
      
        
      
    
      
        
        def
      
      
        finalize(): Unit
      
      
      
- Attributes
 - protected[lang]
 - Definition Classes
 - AnyRef
 - Annotations
 - @throws( classOf[java.lang.Throwable] )
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        getClass(): Class[_]
      
      
      
- Definition Classes
 - AnyRef → Any
 - Annotations
 - @native()
 
 - 
      
      
      
        
      
    
      
        
        def
      
      
        hashCode(): Int
      
      
      
- Definition Classes
 - AnyRef → Any
 - Annotations
 - @native()
 
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectCheckRule(builder: CheckRuleBuilder): Unit
      
      
      
Inject an check analysis
Rulebuilder into the SparkSession.Inject an check analysis
Rulebuilder into the SparkSession. The injected rules will be executed after the analysis phase. A check analysis rule is used to detect problems with a LogicalPlan and should throw an exception when a problem is found. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectColumnar(builder: ColumnarRuleBuilder): Unit
      
      
      
Inject a rule that can override the columnar execution of an executor.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectFunction(functionDescription: FunctionDescription): Unit
      
      
      
Injects a custom function into the org.apache.spark.sql.catalyst.analysis.FunctionRegistry at runtime for all sessions.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectOptimizerRule(builder: RuleBuilder): Unit
      
      
      
Inject an optimizer
Rulebuilder into the SparkSession.Inject an optimizer
Rulebuilder into the SparkSession. The injected rules will be executed during the operator optimization batch. An optimizer rule is used to improve the quality of an analyzed logical plan; these rules should never modify the result of the LogicalPlan. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectParser(builder: ParserBuilder): Unit
      
      
      
Inject a custom parser into the SparkSession.
Inject a custom parser into the SparkSession. Note that the builder is passed a session and an initial parser. The latter allows for a user to create a partial parser and to delegate to the underlying parser for completeness. If a user injects more parsers, then the parsers are stacked on top of each other.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectPlanNormalizationRule(builder: RuleBuilder): Unit
      
      
      
Inject a plan normalization
Rulebuilder into the SparkSession.Inject a plan normalization
Rulebuilder into the SparkSession. The injected rules will be executed just before query caching decisions are made. Such rules can be used to improve the cache hit rate by normalizing different plans to the same form. These rules should never modify the result of the LogicalPlan. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectPlannerStrategy(builder: StrategyBuilder): Unit
      
      
      
Inject a planner
Strategybuilder into the SparkSession.Inject a planner
Strategybuilder into the SparkSession. The injected strategy will be used to convert aLogicalPlaninto a executable org.apache.spark.sql.execution.SparkPlan. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectPostHocResolutionRule(builder: RuleBuilder): Unit
      
      
      
Inject an analyzer
Rulebuilder into the SparkSession.Inject an analyzer
Rulebuilder into the SparkSession. These analyzer rules will be executed after resolution. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectPreCBORule(builder: RuleBuilder): Unit
      
      
      
Inject an optimizer
Rulebuilder that rewrites logical plans into the SparkSession.Inject an optimizer
Rulebuilder that rewrites logical plans into the SparkSession. The injected rules will be executed once after the operator optimization batch and before any cost-based optimization rules that depend on stats. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectQueryStageOptimizerRule(builder: QueryStageOptimizerRuleBuilder): Unit
      
      
      
Inject a rule that can override the query stage optimizer phase of adaptive query execution.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectQueryStagePrepRule(builder: QueryStagePrepRuleBuilder): Unit
      
      
      
Inject a rule that can override the query stage preparation phase of adaptive query execution.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectResolutionRule(builder: RuleBuilder): Unit
      
      
      
Inject an analyzer resolution
Rulebuilder into the SparkSession.Inject an analyzer resolution
Rulebuilder into the SparkSession. These analyzer rules will be executed as part of the resolution phase of analysis. - 
      
      
      
        
      
    
      
        
        def
      
      
        injectRuntimeOptimizerRule(builder: RuleBuilder): Unit
      
      
      
Inject a runtime
Rulebuilder into the SparkSession.Inject a runtime
Rulebuilder into the SparkSession. The injected rules will be executed after built-in org.apache.spark.sql.execution.adaptive.AQEOptimizer rules are applied. A runtime optimizer rule is used to improve the quality of a logical plan during execution which can leverage accurate statistics from shuffle.Note that, it does not work if adaptive query execution is disabled.
 - 
      
      
      
        
      
    
      
        
        def
      
      
        injectTableFunction(functionDescription: TableFunctionDescription): Unit
      
      
      
Injects a custom function into the org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry at runtime for all sessions.
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        isInstanceOf[T0]: Boolean
      
      
      
- Definition Classes
 - Any
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        ne(arg0: AnyRef): Boolean
      
      
      
- Definition Classes
 - AnyRef
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        notify(): Unit
      
      
      
- Definition Classes
 - AnyRef
 - Annotations
 - @native()
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        notifyAll(): Unit
      
      
      
- Definition Classes
 - AnyRef
 - Annotations
 - @native()
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        synchronized[T0](arg0: ⇒ T0): T0
      
      
      
- Definition Classes
 - AnyRef
 
 - 
      
      
      
        
      
    
      
        
        def
      
      
        toString(): String
      
      
      
- Definition Classes
 - AnyRef → Any
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(): Unit
      
      
      
- Definition Classes
 - AnyRef
 - Annotations
 - @throws( ... )
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(arg0: Long, arg1: Int): Unit
      
      
      
- Definition Classes
 - AnyRef
 - Annotations
 - @throws( ... )
 
 - 
      
      
      
        
      
    
      
        final 
        def
      
      
        wait(arg0: Long): Unit
      
      
      
- Definition Classes
 - AnyRef
 - Annotations
 - @throws( ... ) @native()