|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql |
| 19 | + |
| 20 | +import scala.collection.mutable |
| 21 | + |
| 22 | +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} |
| 23 | +import org.apache.spark.sql.catalyst.parser.ParserInterface |
| 24 | +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan |
| 25 | +import org.apache.spark.sql.catalyst.rules.Rule |
| 26 | + |
| 27 | +/** |
| 28 | + * :: Experimental :: |
| 29 | + * Holder for injection points to the [[SparkSession]]. We make NO guarantee about the stability |
| 30 | + * regarding binary compatibility and source compatibility of methods here. |
| 31 | + * |
| 32 | + * This current provides the following extension points: |
| 33 | + * - Analyzer Rules. |
| 34 | + * - Check Analysis Rules |
| 35 | + * - Optimizer Rules. |
| 36 | + * - Planning Strategies. |
| 37 | + * - Customized Parser. |
| 38 | + * - (External) Catalog listeners. |
| 39 | + * |
| 40 | + * The extensions can be used by calling withExtension on the [[SparkSession.Builder]], for |
| 41 | + * example: |
| 42 | + * {{{ |
| 43 | + * SparkSession.builder() |
| 44 | + * .master("...") |
| 45 | + * .conf("...", true) |
| 46 | + * .withExtensions { extensions => |
| 47 | + * extensions.injectResolutionRule { session => |
| 48 | + * ... |
| 49 | + * } |
| 50 | + * extensions.injectParser { (session, parser) => |
| 51 | + * ... |
| 52 | + * } |
| 53 | + * } |
| 54 | + * .getOrCreate() |
| 55 | + * }}} |
| 56 | + * |
| 57 | + * Note that none of the injected builders should assume that the [[SparkSession]] is fully |
| 58 | + * initialized and should not touch the session's internals (e.g. the SessionState). |
| 59 | + */ |
| 60 | +@DeveloperApi |
| 61 | +@Experimental |
| 62 | +@InterfaceStability.Unstable |
| 63 | +class SparkSessionExtensions { |
| 64 | + type RuleBuilder = SparkSession => Rule[LogicalPlan] |
| 65 | + type CheckRuleBuilder = SparkSession => LogicalPlan => Unit |
| 66 | + type StrategyBuilder = SparkSession => Strategy |
| 67 | + type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface |
| 68 | + |
| 69 | + private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] |
| 70 | + |
| 71 | + /** |
| 72 | + * Build the analyzer resolution `Rule`s using the given [[SparkSession]]. |
| 73 | + */ |
| 74 | + private[sql] def buildResolutionRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { |
| 75 | + resolutionRuleBuilders.map(_.apply(session)) |
| 76 | + } |
| 77 | + |
| 78 | + /** |
| 79 | + * Inject an analyzer resolution `Rule` builder into the [[SparkSession]]. These analyzer |
| 80 | + * rules will be executed as part of the resolution phase of analysis. |
| 81 | + */ |
| 82 | + def injectResolutionRule(builder: RuleBuilder): Unit = { |
| 83 | + resolutionRuleBuilders += builder |
| 84 | + } |
| 85 | + |
| 86 | + private[this] val postHocResolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] |
| 87 | + |
| 88 | + /** |
| 89 | + * Build the analyzer post-hoc resolution `Rule`s using the given [[SparkSession]]. |
| 90 | + */ |
| 91 | + private[sql] def buildPostHocResolutionRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { |
| 92 | + postHocResolutionRuleBuilders.map(_.apply(session)) |
| 93 | + } |
| 94 | + |
| 95 | + /** |
| 96 | + * Inject an analyzer `Rule` builder into the [[SparkSession]]. These analyzer |
| 97 | + * rules will be executed after resolution. |
| 98 | + */ |
| 99 | + def injectPostHocResolutionRule(builder: RuleBuilder): Unit = { |
| 100 | + postHocResolutionRuleBuilders += builder |
| 101 | + } |
| 102 | + |
| 103 | + private[this] val checkRuleBuilders = mutable.Buffer.empty[CheckRuleBuilder] |
| 104 | + |
| 105 | + /** |
| 106 | + * Build the check analysis `Rule`s using the given [[SparkSession]]. |
| 107 | + */ |
| 108 | + private[sql] def buildCheckRules(session: SparkSession): Seq[LogicalPlan => Unit] = { |
| 109 | + checkRuleBuilders.map(_.apply(session)) |
| 110 | + } |
| 111 | + |
| 112 | + /** |
| 113 | + * Inject an check analysis `Rule` builder into the [[SparkSession]]. The injected rules will |
| 114 | + * be executed after the analysis phase. A check analysis rule is used to detect problems with a |
| 115 | + * LogicalPlan and should throw an exception when a problem is found. |
| 116 | + */ |
| 117 | + def injectCheckRule(builder: CheckRuleBuilder): Unit = { |
| 118 | + checkRuleBuilders += builder |
| 119 | + } |
| 120 | + |
| 121 | + private[this] val optimizerRules = mutable.Buffer.empty[RuleBuilder] |
| 122 | + |
| 123 | + private[sql] def buildOptimizerRules(session: SparkSession): Seq[Rule[LogicalPlan]] = { |
| 124 | + optimizerRules.map(_.apply(session)) |
| 125 | + } |
| 126 | + |
| 127 | + /** |
| 128 | + * Inject an optimizer `Rule` builder into the [[SparkSession]]. The injected rules will be |
| 129 | + * executed during the operator optimization batch. An optimizer rule is used to improve the |
| 130 | + * quality of an analyzed logical plan; these rules should never modify the result of the |
| 131 | + * LogicalPlan. |
| 132 | + */ |
| 133 | + def injectOptimizerRule(builder: RuleBuilder): Unit = { |
| 134 | + optimizerRules += builder |
| 135 | + } |
| 136 | + |
| 137 | + private[this] val plannerStrategyBuilders = mutable.Buffer.empty[StrategyBuilder] |
| 138 | + |
| 139 | + private[sql] def buildPlannerStrategies(session: SparkSession): Seq[Strategy] = { |
| 140 | + plannerStrategyBuilders.map(_.apply(session)) |
| 141 | + } |
| 142 | + |
| 143 | + /** |
| 144 | + * Inject a planner `Strategy` builder into the [[SparkSession]]. The injected strategy will |
| 145 | + * be used to convert a `LogicalPlan` into a executable |
| 146 | + * [[org.apache.spark.sql.execution.SparkPlan]]. |
| 147 | + */ |
| 148 | + def injectPlannerStrategy(builder: StrategyBuilder): Unit = { |
| 149 | + plannerStrategyBuilders += builder |
| 150 | + } |
| 151 | + |
| 152 | + private[this] val parserBuilders = mutable.Buffer.empty[ParserBuilder] |
| 153 | + |
| 154 | + private[sql] def buildParser( |
| 155 | + session: SparkSession, |
| 156 | + initial: ParserInterface): ParserInterface = { |
| 157 | + parserBuilders.foldLeft(initial) { (parser, builder) => |
| 158 | + builder(session, parser) |
| 159 | + } |
| 160 | + } |
| 161 | + |
| 162 | + /** |
| 163 | + * Inject a custom parser into the [[SparkSession]]. Note that the builder is passed a session |
| 164 | + * and an initial parser. The latter allows for a user to create a partial parser and to delegate |
| 165 | + * to the underlying parser for completeness. If a user injects more parsers, then the parsers |
| 166 | + * are stacked on top of each other. |
| 167 | + */ |
| 168 | + def injectParser(builder: ParserBuilder): Unit = { |
| 169 | + parserBuilders += builder |
| 170 | + } |
| 171 | +} |
0 commit comments