Skip to content

Commit b2dfa84

Browse files
yhuairxin
authored andcommitted
[SPARK-14668][SQL] Move CurrentDatabase to Catalyst
## What changes were proposed in this pull request? This PR moves `CurrentDatabase` from sql/hive package to sql/catalyst. It also adds the function description, which looks like the following. ``` scala> sqlContext.sql("describe function extended current_database").collect.foreach(println) [Function: current_database] [Class: org.apache.spark.sql.execution.command.CurrentDatabase] [Usage: current_database() - Returns the current database.] [Extended Usage: > SELECT current_database()] ``` ## How was this patch tested? Existing tests Author: Yin Huai <[email protected]> Closes #12424 from yhuai/SPARK-14668.
1 parent 4df6518 commit b2dfa84

File tree

7 files changed

+47
-24
lines changed

7 files changed

+47
-24
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -337,6 +337,7 @@ object FunctionRegistry {
337337
expression[SparkPartitionID]("spark_partition_id"),
338338
expression[InputFileName]("input_file_name"),
339339
expression[MonotonicallyIncreasingID]("monotonically_increasing_id"),
340+
expression[CurrentDatabase]("current_database"),
340341

341342
// grouping sets
342343
expression[Cube]("cube"),

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -512,3 +512,15 @@ object XxHash64Function extends InterpretedHashFunction {
512512
XXH64.hashUnsafeBytes(base, offset, len, seed)
513513
}
514514
}
515+
516+
/**
517+
* Returns the current database of the SessionCatalog.
518+
*/
519+
@ExpressionDescription(
520+
usage = "_FUNC_() - Returns the current database.",
521+
extended = "> SELECT _FUNC_()")
522+
private[sql] case class CurrentDatabase() extends LeafExpression with Unevaluable {
523+
override def dataType: DataType = StringType
524+
override def foldable: Boolean = true
525+
override def nullable: Boolean = false
526+
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala

Lines changed: 21 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst.optimizer
2020
import scala.annotation.tailrec
2121
import scala.collection.immutable.HashSet
2222

23-
import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases}
23+
import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf}
24+
import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry}
25+
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
2426
import org.apache.spark.sql.catalyst.expressions._
2527
import org.apache.spark.sql.catalyst.expressions.aggregate._
2628
import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
@@ -34,7 +36,9 @@ import org.apache.spark.sql.types._
3436
* Abstract class all optimizers should inherit of, contains the standard batches (extending
3537
* Optimizers can override this.
3638
*/
37-
abstract class Optimizer extends RuleExecutor[LogicalPlan] {
39+
abstract class Optimizer(
40+
conf: CatalystConf,
41+
sessionCatalog: SessionCatalog) extends RuleExecutor[LogicalPlan] {
3842
def batches: Seq[Batch] = {
3943
// Technically some of the rules in Finish Analysis are not optimizer rules and belong more
4044
// in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime).
@@ -43,6 +47,7 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
4347
Batch("Finish Analysis", Once,
4448
EliminateSubqueryAliases,
4549
ComputeCurrentTime,
50+
GetCurrentDatabase(sessionCatalog),
4651
DistinctAggregationRewriter) ::
4752
//////////////////////////////////////////////////////////////////////////////////////////
4853
// Optimizer rules start here
@@ -117,7 +122,10 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] {
117122
* To ensure extendability, we leave the standard rules in the abstract optimizer rules, while
118123
* specific rules go to the subclasses
119124
*/
120-
object DefaultOptimizer extends Optimizer
125+
object DefaultOptimizer
126+
extends Optimizer(
127+
EmptyConf,
128+
new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, EmptyConf))
121129

122130
/**
123131
* Pushes operations down into a Sample.
@@ -1399,6 +1407,16 @@ object ComputeCurrentTime extends Rule[LogicalPlan] {
13991407
}
14001408
}
14011409

1410+
/** Replaces the expression of CurrentDatabase with the current database name. */
1411+
case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[LogicalPlan] {
1412+
def apply(plan: LogicalPlan): LogicalPlan = {
1413+
plan transformAllExpressions {
1414+
case CurrentDatabase() =>
1415+
Literal.create(sessionCatalog.getCurrentDatabase, StringType)
1416+
}
1417+
}
1418+
}
1419+
14021420
/**
14031421
* Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a
14041422
* [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerExtendableSuite.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.spark.sql.catalyst
1919

2020
import org.apache.spark.SparkFunSuite
21+
import org.apache.spark.sql.catalyst.analysis.EmptyFunctionRegistry
22+
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
2123
import org.apache.spark.sql.catalyst.optimizer.Optimizer
2224
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2325
import org.apache.spark.sql.catalyst.rules.Rule
@@ -38,7 +40,10 @@ class OptimizerExtendableSuite extends SparkFunSuite {
3840
* This class represents a dummy extended optimizer that takes the batches of the
3941
* Optimizer and adds custom ones.
4042
*/
41-
class ExtendedOptimizer extends Optimizer {
43+
class ExtendedOptimizer
44+
extends Optimizer(
45+
EmptyConf,
46+
new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, EmptyConf)) {
4247

4348
// rules set to DummyRule, would not be executed anyways
4449
val myBatches: Seq[Batch] = {

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,14 @@
1818
package org.apache.spark.sql.execution
1919

2020
import org.apache.spark.sql.ExperimentalMethods
21+
import org.apache.spark.sql.catalyst.CatalystConf
22+
import org.apache.spark.sql.catalyst.catalog.SessionCatalog
2123
import org.apache.spark.sql.catalyst.optimizer.Optimizer
2224

23-
class SparkOptimizer(experimentalMethods: ExperimentalMethods) extends Optimizer {
25+
class SparkOptimizer(
26+
conf: CatalystConf,
27+
sessionCatalog: SessionCatalog,
28+
experimentalMethods: ExperimentalMethods) extends Optimizer(conf, sessionCatalog) {
2429
override def batches: Seq[Batch] = super.batches :+ Batch(
2530
"User Provided Optimizers", FixedPoint(100), experimentalMethods.extraOptimizations: _*)
2631
}

sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ private[sql] class SessionState(ctx: SQLContext) {
8080
/**
8181
* Logical query plan optimizer.
8282
*/
83-
lazy val optimizer: Optimizer = new SparkOptimizer(experimentalMethods)
83+
lazy val optimizer: Optimizer = new SparkOptimizer(conf, catalog, experimentalMethods)
8484

8585
/**
8686
* Parser that extracts expressions, plans, table identifiers etc. from SQL texts.

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala

Lines changed: 0 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -60,19 +60,6 @@ import org.apache.spark.sql.types._
6060
import org.apache.spark.unsafe.types.UTF8String
6161
import org.apache.spark.util.Utils
6262

63-
/**
64-
* Returns the current database of metadataHive.
65-
*/
66-
private[hive] case class CurrentDatabase(ctx: HiveContext)
67-
extends LeafExpression with CodegenFallback {
68-
override def dataType: DataType = StringType
69-
override def foldable: Boolean = true
70-
override def nullable: Boolean = false
71-
override def eval(input: InternalRow): Any = {
72-
UTF8String.fromString(ctx.sessionState.catalog.getCurrentDatabase)
73-
}
74-
}
75-
7663
/**
7764
* An instance of the Spark SQL execution engine that integrates with data stored in Hive.
7865
* Configuration for Hive is read from hive-site.xml on the classpath.
@@ -133,11 +120,6 @@ class HiveContext private[hive](
133120
@transient
134121
protected[sql] override lazy val sessionState = new HiveSessionState(self)
135122

136-
// The Hive UDF current_database() is foldable, will be evaluated by optimizer,
137-
// but the optimizer can't access the SessionState of metadataHive.
138-
sessionState.functionRegistry.registerFunction(
139-
"current_database", (e: Seq[Expression]) => new CurrentDatabase(self))
140-
141123
/**
142124
* When true, enables an experimental feature where metastore tables that use the parquet SerDe
143125
* are automatically converted to use the Spark SQL parquet table scan, instead of the Hive

0 commit comments

Comments
 (0)