diff options
author | Reynold Xin <rxin@databricks.com> | 2016-04-25 16:20:57 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-04-25 16:20:57 -0700 |
commit | f36c9c83798877256efa1447a6b9be5aa47a7e87 (patch) | |
tree | e5b06cc8944e34dd1193793bf706258441e599f2 /sql/hive | |
parent | 34336b6250d99bcf009b082cbf83f326d6b00074 (diff) | |
download | spark-f36c9c83798877256efa1447a6b9be5aa47a7e87.tar.gz spark-f36c9c83798877256efa1447a6b9be5aa47a7e87.tar.bz2 spark-f36c9c83798877256efa1447a6b9be5aa47a7e87.zip |
[SPARK-14888][SQL] UnresolvedFunction should use FunctionIdentifier
## What changes were proposed in this pull request?
This patch changes UnresolvedFunction and UnresolvedGenerator to use a FunctionIdentifier rather than just a String for function name. Also changed SessionCatalog to accept FunctionIdentifier in lookupFunction.
## How was this patch tested?
Updated related unit tests.
Author: Reynold Xin <rxin@databricks.com>
Closes #12659 from rxin/SPARK-14888.
Diffstat (limited to 'sql/hive')
2 files changed, 7 insertions, 9 deletions
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 3e97edb67e..01b24d2f7e 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -74,7 +74,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) TestHive.sessionState.functionRegistry.restore() - // For debugging dump some statistics about how much time was spent in various optimizer rules. + // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) } finally { super.afterAll() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 3e718826fe..9e527073d4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -27,18 +27,16 @@ import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} import org.apache.spark.sql.{AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -176,7 +174,7 @@ private[sql] class HiveSessionCatalog( // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean, // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number, // xpath_short, and xpath_string. - override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + override def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { // TODO: Once lookupFunction accepts a FunctionIdentifier, we should refactor this method to // if (super.functionExists(name)) { // super.lookupFunction(name, children) @@ -187,7 +185,7 @@ private[sql] class HiveSessionCatalog( Try(super.lookupFunction(name, children)) match { case Success(expr) => expr case Failure(error) => - if (functionRegistry.functionExists(name)) { + if (functionRegistry.functionExists(name.unquotedString)) { // If the function actually exists in functionRegistry, it means that there is an // error when we create the Expression using the given children. // We need to throw the original exception. @@ -196,7 +194,7 @@ private[sql] class HiveSessionCatalog( // This function is not in functionRegistry, let's try to load it as a Hive's // built-in function. // Hive is case insensitive. - val functionName = name.toLowerCase + val functionName = name.unquotedString.toLowerCase // TODO: This may not really work for current_user because current_user is not evaluated // with session info. // We do not need to use executionHive at here because we only load @@ -204,12 +202,12 @@ private[sql] class HiveSessionCatalog( val functionInfo = { try { Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( - failFunctionLookup(name)) + failFunctionLookup(name.unquotedString)) } catch { // If HiveFunctionRegistry.getFunctionInfo throws an exception, // we are failing to load a Hive builtin function, which means that // the given function is not a Hive builtin function. - case NonFatal(e) => failFunctionLookup(name) + case NonFatal(e) => failFunctionLookup(name.unquotedString) } } val className = functionInfo.getFunctionClass.getName |