diff --git ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 9ac540e..2621370 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -423,6 +423,7 @@ "sorted, table {0}", true), ALTER_TABLE_TYPE_PARTIAL_PARTITION_SPEC_NO_SUPPORTED(10299, "Alter table partition type {0} does not allow partial partition spec"), + DROP_NATIVE_FUNCTION(10300, "Cannot drop native function"), //========================== 20000 range starts here ========================// SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."), diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java deleted file mode 100644 index 93c15c0..0000000 --- ql/src/java/org/apache/hadoop/hive/ql/exec/CommonFunctionInfo.java +++ /dev/null @@ -1,25 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.exec; - -/** - * Interface for common functionality between FunctionInfo/WindowFunctionInfo - */ -public interface CommonFunctionInfo { - Class getFunctionClass(); -} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java index 074255b..07ad19b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionInfo.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec; +import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFBridge; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; @@ -27,18 +28,26 @@ import org.apache.hadoop.hive.ql.udf.ptf.WindowingTableFunction; import org.apache.hive.common.util.AnnotationUtils; +import java.util.List; + /** * FunctionInfo. * */ -public class FunctionInfo implements CommonFunctionInfo { +public class FunctionInfo { private final boolean isNative; + private final boolean isPersistent; + private final boolean isInternalTableFunction; private final String displayName; + private final List resources; + + private String className; + private GenericUDF genericUDF; private GenericUDTF genericUDTF; @@ -47,38 +56,55 @@ private Class tableFunctionResolver; + public FunctionInfo(String displayName, String className, List resources) { + this.isNative = true; + this.isPersistent = true; + this.displayName = displayName; + this.className = className; + this.isInternalTableFunction = false; + this.resources = resources; + } + public FunctionInfo(boolean isNative, String displayName, - GenericUDF genericUDF) { + GenericUDF genericUDF, List resources) { this.isNative = isNative; + this.isPersistent = false; this.displayName = displayName; this.genericUDF = genericUDF; this.isInternalTableFunction = false; + this.resources = resources; } public FunctionInfo(boolean isNative, String displayName, - GenericUDAFResolver genericUDAFResolver) { + GenericUDAFResolver genericUDAFResolver, List resources) { this.isNative = isNative; + this.isPersistent = false; this.displayName = displayName; this.genericUDAFResolver = genericUDAFResolver; this.isInternalTableFunction = false; + this.resources = resources; } public FunctionInfo(boolean isNative, String displayName, - GenericUDTF genericUDTF) { + GenericUDTF genericUDTF, List resources) { this.isNative = isNative; + this.isPersistent = false; this.displayName = displayName; this.genericUDTF = genericUDTF; this.isInternalTableFunction = false; + this.resources = resources; } - public FunctionInfo(String displayName, Class tFnCls) - { + public FunctionInfo(String displayName, Class tFnCls, + List resources) { + this.isPersistent = false; this.displayName = displayName; this.tableFunctionResolver = tFnCls; PartitionTableFunctionDescription def = AnnotationUtils.getAnnotation( tableFunctionResolver, PartitionTableFunctionDescription.class); - this.isNative = (def == null) ? false : def.isInternal(); + this.isNative = def != null && def.isInternal(); this.isInternalTableFunction = isNative; + this.resources = resources; } /** @@ -190,4 +216,16 @@ public boolean isGenericUDTF() { public boolean isTableFunction() { return null != tableFunctionResolver; } + + public boolean isPersistent() { + return isPersistent; + } + + public String getClassName() { + return className; + } + + public List getResources() { + return resources; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java index b94f790..40cf9a5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java @@ -18,33 +18,22 @@ package org.apache.hadoop.hive.ql.exec; -import java.io.InputStream; import java.lang.reflect.Method; -import java.net.URL; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.TreeSet; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Function; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.ql.exec.FunctionUtils.UDFClassType; +import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -141,27 +130,16 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hive.common.util.AnnotationUtils; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; - /** * FunctionRegistry. */ public final class FunctionRegistry { - private static Log LOG = LogFactory.getLog(FunctionRegistry.class); - - /** - * The mapping from expression function names to expression classes. - */ - static Map mFunctions = Collections.synchronizedMap(new LinkedHashMap()); + private static final Log LOG = LogFactory.getLog(FunctionRegistry.class); - static Set> nativeUdfs = Collections.synchronizedSet(new HashSet>()); /* * PTF variables * */ @@ -176,557 +154,331 @@ private static final String NOOP_MAP_TABLE_FUNCTION = "noopwithmap"; private static final String NOOP_STREAMING_TABLE_FUNCTION = "noopstreaming"; private static final String NOOP_STREAMING_MAP_TABLE_FUNCTION = "noopwithmapstreaming"; + private static final String MATCH_PATH_TABLE_FUNCTION = "matchpath"; - static Map windowFunctions = Collections.synchronizedMap(new LinkedHashMap()); + // registry for system functions + private static final Registry system = new Registry(true); static { - registerGenericUDF("concat", GenericUDFConcat.class); - registerUDF("substr", UDFSubstr.class, false); - registerUDF("substring", UDFSubstr.class, false); - registerUDF("space", UDFSpace.class, false); - registerUDF("repeat", UDFRepeat.class, false); - registerUDF("ascii", UDFAscii.class, false); - registerGenericUDF("lpad", GenericUDFLpad.class); - registerGenericUDF("rpad", GenericUDFRpad.class); - - registerGenericUDF("size", GenericUDFSize.class); - - registerGenericUDF("round", GenericUDFRound.class); - registerGenericUDF("floor", GenericUDFFloor.class); - registerUDF("sqrt", UDFSqrt.class, false); - registerGenericUDF("ceil", GenericUDFCeil.class); - registerGenericUDF("ceiling", GenericUDFCeil.class); - registerUDF("rand", UDFRand.class, false); - registerGenericUDF("abs", GenericUDFAbs.class); - registerGenericUDF("pmod", GenericUDFPosMod.class); - - registerUDF("ln", UDFLn.class, false); - registerUDF("log2", UDFLog2.class, false); - registerUDF("sin", UDFSin.class, false); - registerUDF("asin", UDFAsin.class, false); - registerUDF("cos", UDFCos.class, false); - registerUDF("acos", UDFAcos.class, false); - registerUDF("log10", UDFLog10.class, false); - registerUDF("log", UDFLog.class, false); - registerUDF("exp", UDFExp.class, false); - registerGenericUDF("power", GenericUDFPower.class); - registerGenericUDF("pow", GenericUDFPower.class); - registerUDF("sign", UDFSign.class, false); - registerUDF("pi", UDFPI.class, false); - registerUDF("degrees", UDFDegrees.class, false); - registerUDF("radians", UDFRadians.class, false); - registerUDF("atan", UDFAtan.class, false); - registerUDF("tan", UDFTan.class, false); - registerUDF("e", UDFE.class, false); - - registerUDF("conv", UDFConv.class, false); - registerUDF("bin", UDFBin.class, false); - registerUDF("hex", UDFHex.class, false); - registerUDF("unhex", UDFUnhex.class, false); - registerUDF("base64", UDFBase64.class, false); - registerUDF("unbase64", UDFUnbase64.class, false); - - registerGenericUDF("encode", GenericUDFEncode.class); - registerGenericUDF("decode", GenericUDFDecode.class); - - registerGenericUDF("upper", GenericUDFUpper.class); - registerGenericUDF("lower", GenericUDFLower.class); - registerGenericUDF("ucase", GenericUDFUpper.class); - registerGenericUDF("lcase", GenericUDFLower.class); - registerGenericUDF("trim", GenericUDFTrim.class); - registerGenericUDF("ltrim", GenericUDFLTrim.class); - registerGenericUDF("rtrim", GenericUDFRTrim.class); - registerUDF("length", UDFLength.class, false); - registerUDF("reverse", UDFReverse.class, false); - registerGenericUDF("field", GenericUDFField.class); - registerUDF("find_in_set", UDFFindInSet.class, false); - - registerUDF("like", UDFLike.class, true); - registerUDF("rlike", UDFRegExp.class, true); - registerUDF("regexp", UDFRegExp.class, true); - registerUDF("regexp_replace", UDFRegExpReplace.class, false); - registerUDF("regexp_extract", UDFRegExpExtract.class, false); - registerUDF("parse_url", UDFParseUrl.class, false); - registerGenericUDF("nvl", GenericUDFNvl.class); - registerGenericUDF("split", GenericUDFSplit.class); - registerGenericUDF("str_to_map", GenericUDFStringToMap.class); - registerGenericUDF("translate", GenericUDFTranslate.class); - - registerGenericUDF("positive", GenericUDFOPPositive.class); - registerGenericUDF("negative", GenericUDFOPNegative.class); - - registerUDF("day", UDFDayOfMonth.class, false); - registerUDF("dayofmonth", UDFDayOfMonth.class, false); - registerUDF("month", UDFMonth.class, false); - registerUDF("year", UDFYear.class, false); - registerUDF("hour", UDFHour.class, false); - registerUDF("minute", UDFMinute.class, false); - registerUDF("second", UDFSecond.class, false); - registerUDF("from_unixtime", UDFFromUnixTime.class, false); - registerGenericUDF("to_date", GenericUDFDate.class); - registerUDF("weekofyear", UDFWeekOfYear.class, false); - - registerGenericUDF("date_add", GenericUDFDateAdd.class); - registerGenericUDF("date_sub", GenericUDFDateSub.class); - registerGenericUDF("datediff", GenericUDFDateDiff.class); - - registerUDF("get_json_object", UDFJson.class, false); - - registerUDF("xpath_string", UDFXPathString.class, false); - registerUDF("xpath_boolean", UDFXPathBoolean.class, false); - registerUDF("xpath_number", UDFXPathDouble.class, false); - registerUDF("xpath_double", UDFXPathDouble.class, false); - registerUDF("xpath_float", UDFXPathFloat.class, false); - registerUDF("xpath_long", UDFXPathLong.class, false); - registerUDF("xpath_int", UDFXPathInteger.class, false); - registerUDF("xpath_short", UDFXPathShort.class, false); - registerGenericUDF("xpath", GenericUDFXPath.class); - - registerGenericUDF("+", GenericUDFOPPlus.class); - registerGenericUDF("-", GenericUDFOPMinus.class); - registerGenericUDF("*", GenericUDFOPMultiply.class); - registerGenericUDF("/", GenericUDFOPDivide.class); - registerGenericUDF("%", GenericUDFOPMod.class); - registerUDF("div", UDFOPLongDivide.class, true); - - registerUDF("&", UDFOPBitAnd.class, true); - registerUDF("|", UDFOPBitOr.class, true); - registerUDF("^", UDFOPBitXor.class, true); - registerUDF("~", UDFOPBitNot.class, true); - - registerGenericUDF("current_database", UDFCurrentDB.class); - - registerGenericUDF("isnull", GenericUDFOPNull.class); - registerGenericUDF("isnotnull", GenericUDFOPNotNull.class); - - registerGenericUDF("if", GenericUDFIf.class); - registerGenericUDF("in", GenericUDFIn.class); - registerGenericUDF("and", GenericUDFOPAnd.class); - registerGenericUDF("or", GenericUDFOPOr.class); - registerGenericUDF("=", GenericUDFOPEqual.class); - registerGenericUDF("==", GenericUDFOPEqual.class); - registerGenericUDF("<=>", GenericUDFOPEqualNS.class); - registerGenericUDF("!=", GenericUDFOPNotEqual.class); - registerGenericUDF("<>", GenericUDFOPNotEqual.class); - registerGenericUDF("<", GenericUDFOPLessThan.class); - registerGenericUDF("<=", GenericUDFOPEqualOrLessThan.class); - registerGenericUDF(">", GenericUDFOPGreaterThan.class); - registerGenericUDF(">=", GenericUDFOPEqualOrGreaterThan.class); - registerGenericUDF("not", GenericUDFOPNot.class); - registerGenericUDF("!", GenericUDFOPNot.class); - registerGenericUDF("between", GenericUDFBetween.class); - - registerGenericUDF("ewah_bitmap_and", GenericUDFEWAHBitmapAnd.class); - registerGenericUDF("ewah_bitmap_or", GenericUDFEWAHBitmapOr.class); - registerGenericUDF("ewah_bitmap_empty", GenericUDFEWAHBitmapEmpty.class); + system.registerGenericUDF("concat", GenericUDFConcat.class, null); + system.registerUDF("substr", UDFSubstr.class, false, null); + system.registerUDF("substring", UDFSubstr.class, false, null); + system.registerUDF("space", UDFSpace.class, false, null); + system.registerUDF("repeat", UDFRepeat.class, false, null); + system.registerUDF("ascii", UDFAscii.class, false, null); + system.registerGenericUDF("lpad", GenericUDFLpad.class, null); + system.registerGenericUDF("rpad", GenericUDFRpad.class, null); + + system.registerGenericUDF("size", GenericUDFSize.class, null); + + system.registerGenericUDF("round", GenericUDFRound.class, null); + system.registerGenericUDF("floor", GenericUDFFloor.class, null); + system.registerUDF("sqrt", UDFSqrt.class, false, null); + system.registerGenericUDF("ceil", GenericUDFCeil.class, null); + system.registerGenericUDF("ceiling", GenericUDFCeil.class, null); + system.registerUDF("rand", UDFRand.class, false, null); + system.registerGenericUDF("abs", GenericUDFAbs.class, null); + system.registerGenericUDF("pmod", GenericUDFPosMod.class, null); + + system.registerUDF("ln", UDFLn.class, false, null); + system.registerUDF("log2", UDFLog2.class, false, null); + system.registerUDF("sin", UDFSin.class, false, null); + system.registerUDF("asin", UDFAsin.class, false, null); + system.registerUDF("cos", UDFCos.class, false, null); + system.registerUDF("acos", UDFAcos.class, false, null); + system.registerUDF("log10", UDFLog10.class, false, null); + system.registerUDF("log", UDFLog.class, false, null); + system.registerUDF("exp", UDFExp.class, false, null); + system.registerGenericUDF("power", GenericUDFPower.class, null); + system.registerGenericUDF("pow", GenericUDFPower.class, null); + system.registerUDF("sign", UDFSign.class, false, null); + system.registerUDF("pi", UDFPI.class, false, null); + system.registerUDF("degrees", UDFDegrees.class, false, null); + system.registerUDF("radians", UDFRadians.class, false, null); + system.registerUDF("atan", UDFAtan.class, false, null); + system.registerUDF("tan", UDFTan.class, false, null); + system.registerUDF("e", UDFE.class, false, null); + + system.registerUDF("conv", UDFConv.class, false, null); + system.registerUDF("bin", UDFBin.class, false, null); + system.registerUDF("hex", UDFHex.class, false, null); + system.registerUDF("unhex", UDFUnhex.class, false, null); + system.registerUDF("base64", UDFBase64.class, false, null); + system.registerUDF("unbase64", UDFUnbase64.class, false, null); + + system.registerGenericUDF("encode", GenericUDFEncode.class, null); + system.registerGenericUDF("decode", GenericUDFDecode.class, null); + + system.registerGenericUDF("upper", GenericUDFUpper.class, null); + system.registerGenericUDF("lower", GenericUDFLower.class, null); + system.registerGenericUDF("ucase", GenericUDFUpper.class, null); + system.registerGenericUDF("lcase", GenericUDFLower.class, null); + system.registerGenericUDF("trim", GenericUDFTrim.class, null); + system.registerGenericUDF("ltrim", GenericUDFLTrim.class, null); + system.registerGenericUDF("rtrim", GenericUDFRTrim.class, null); + system.registerUDF("length", UDFLength.class, false, null); + system.registerUDF("reverse", UDFReverse.class, false, null); + system.registerGenericUDF("field", GenericUDFField.class, null); + system.registerUDF("find_in_set", UDFFindInSet.class, false, null); + + system.registerUDF("like", UDFLike.class, true, null); + system.registerUDF("rlike", UDFRegExp.class, true, null); + system.registerUDF("regexp", UDFRegExp.class, true, null); + system.registerUDF("regexp_replace", UDFRegExpReplace.class, false, null); + system.registerUDF("regexp_extract", UDFRegExpExtract.class, false, null); + system.registerUDF("parse_url", UDFParseUrl.class, false, null); + system.registerGenericUDF("nvl", GenericUDFNvl.class, null); + system.registerGenericUDF("split", GenericUDFSplit.class, null); + system.registerGenericUDF("str_to_map", GenericUDFStringToMap.class, null); + system.registerGenericUDF("translate", GenericUDFTranslate.class, null); + + system.registerGenericUDF("positive", GenericUDFOPPositive.class, null); + system.registerGenericUDF("negative", GenericUDFOPNegative.class, null); + + system.registerUDF("day", UDFDayOfMonth.class, false, null); + system.registerUDF("dayofmonth", UDFDayOfMonth.class, false, null); + system.registerUDF("month", UDFMonth.class, false, null); + system.registerUDF("year", UDFYear.class, false, null); + system.registerUDF("hour", UDFHour.class, false, null); + system.registerUDF("minute", UDFMinute.class, false, null); + system.registerUDF("second", UDFSecond.class, false, null); + system.registerUDF("from_unixtime", UDFFromUnixTime.class, false, null); + system.registerGenericUDF("to_date", GenericUDFDate.class, null); + system.registerUDF("weekofyear", UDFWeekOfYear.class, false, null); + + system.registerGenericUDF("date_add", GenericUDFDateAdd.class, null); + system.registerGenericUDF("date_sub", GenericUDFDateSub.class, null); + system.registerGenericUDF("datediff", GenericUDFDateDiff.class, null); + + system.registerUDF("get_json_object", UDFJson.class, false, null); + + system.registerUDF("xpath_string", UDFXPathString.class, false, null); + system.registerUDF("xpath_boolean", UDFXPathBoolean.class, false, null); + system.registerUDF("xpath_number", UDFXPathDouble.class, false, null); + system.registerUDF("xpath_double", UDFXPathDouble.class, false, null); + system.registerUDF("xpath_float", UDFXPathFloat.class, false, null); + system.registerUDF("xpath_long", UDFXPathLong.class, false, null); + system.registerUDF("xpath_int", UDFXPathInteger.class, false, null); + system.registerUDF("xpath_short", UDFXPathShort.class, false, null); + system.registerGenericUDF("xpath", GenericUDFXPath.class, null); + + system.registerGenericUDF("+", GenericUDFOPPlus.class, null); + system.registerGenericUDF("-", GenericUDFOPMinus.class, null); + system.registerGenericUDF("*", GenericUDFOPMultiply.class, null); + system.registerGenericUDF("/", GenericUDFOPDivide.class, null); + system.registerGenericUDF("%", GenericUDFOPMod.class, null); + system.registerUDF("div", UDFOPLongDivide.class, true, null); + + system.registerUDF("&", UDFOPBitAnd.class, true, null); + system.registerUDF("|", UDFOPBitOr.class, true, null); + system.registerUDF("^", UDFOPBitXor.class, true, null); + system.registerUDF("~", UDFOPBitNot.class, true, null); + + system.registerGenericUDF("current_database", UDFCurrentDB.class, null); + + system.registerGenericUDF("isnull", GenericUDFOPNull.class, null); + system.registerGenericUDF("isnotnull", GenericUDFOPNotNull.class, null); + + system.registerGenericUDF("if", GenericUDFIf.class, null); + system.registerGenericUDF("in", GenericUDFIn.class, null); + system.registerGenericUDF("and", GenericUDFOPAnd.class, null); + system.registerGenericUDF("or", GenericUDFOPOr.class, null); + system.registerGenericUDF("=", GenericUDFOPEqual.class, null); + system.registerGenericUDF("==", GenericUDFOPEqual.class, null); + system.registerGenericUDF("<=>", GenericUDFOPEqualNS.class, null); + system.registerGenericUDF("!=", GenericUDFOPNotEqual.class, null); + system.registerGenericUDF("<>", GenericUDFOPNotEqual.class, null); + system.registerGenericUDF("<", GenericUDFOPLessThan.class, null); + system.registerGenericUDF("<=", GenericUDFOPEqualOrLessThan.class, null); + system.registerGenericUDF(">", GenericUDFOPGreaterThan.class, null); + system.registerGenericUDF(">=", GenericUDFOPEqualOrGreaterThan.class, null); + system.registerGenericUDF("not", GenericUDFOPNot.class, null); + system.registerGenericUDF("!", GenericUDFOPNot.class, null); + system.registerGenericUDF("between", GenericUDFBetween.class, null); + + system.registerGenericUDF("ewah_bitmap_and", GenericUDFEWAHBitmapAnd.class, null); + system.registerGenericUDF("ewah_bitmap_or", GenericUDFEWAHBitmapOr.class, null); + system.registerGenericUDF("ewah_bitmap_empty", GenericUDFEWAHBitmapEmpty.class, null); // Aliases for Java Class Names // These are used in getImplicitConvertUDFMethod - registerUDF(serdeConstants.BOOLEAN_TYPE_NAME, UDFToBoolean.class, false, - UDFToBoolean.class.getSimpleName()); - registerUDF(serdeConstants.TINYINT_TYPE_NAME, UDFToByte.class, false, - UDFToByte.class.getSimpleName()); - registerUDF(serdeConstants.SMALLINT_TYPE_NAME, UDFToShort.class, false, - UDFToShort.class.getSimpleName()); - registerUDF(serdeConstants.INT_TYPE_NAME, UDFToInteger.class, false, - UDFToInteger.class.getSimpleName()); - registerUDF(serdeConstants.BIGINT_TYPE_NAME, UDFToLong.class, false, - UDFToLong.class.getSimpleName()); - registerUDF(serdeConstants.FLOAT_TYPE_NAME, UDFToFloat.class, false, - UDFToFloat.class.getSimpleName()); - registerUDF(serdeConstants.DOUBLE_TYPE_NAME, UDFToDouble.class, false, - UDFToDouble.class.getSimpleName()); - registerUDF(serdeConstants.STRING_TYPE_NAME, UDFToString.class, false, - UDFToString.class.getSimpleName()); - - registerGenericUDF(serdeConstants.DATE_TYPE_NAME, - GenericUDFToDate.class); - registerGenericUDF(serdeConstants.TIMESTAMP_TYPE_NAME, - GenericUDFTimestamp.class); - registerGenericUDF(serdeConstants.BINARY_TYPE_NAME, - GenericUDFToBinary.class); - registerGenericUDF(serdeConstants.DECIMAL_TYPE_NAME, - GenericUDFToDecimal.class); - registerGenericUDF(serdeConstants.VARCHAR_TYPE_NAME, - GenericUDFToVarchar.class); - registerGenericUDF(serdeConstants.CHAR_TYPE_NAME, - GenericUDFToChar.class); + system.registerUDF(serdeConstants.BOOLEAN_TYPE_NAME, UDFToBoolean.class, false, UDFToBoolean.class.getSimpleName(), null); + system.registerUDF(serdeConstants.TINYINT_TYPE_NAME, UDFToByte.class, false, UDFToByte.class.getSimpleName(), null); + system.registerUDF(serdeConstants.SMALLINT_TYPE_NAME, UDFToShort.class, false, UDFToShort.class.getSimpleName(), null); + system.registerUDF(serdeConstants.INT_TYPE_NAME, UDFToInteger.class, false, UDFToInteger.class.getSimpleName(), null); + system.registerUDF(serdeConstants.BIGINT_TYPE_NAME, UDFToLong.class, false, UDFToLong.class.getSimpleName(), null); + system.registerUDF(serdeConstants.FLOAT_TYPE_NAME, UDFToFloat.class, false, UDFToFloat.class.getSimpleName(), null); + system.registerUDF(serdeConstants.DOUBLE_TYPE_NAME, UDFToDouble.class, false, UDFToDouble.class.getSimpleName(), null); + system.registerUDF(serdeConstants.STRING_TYPE_NAME, UDFToString.class, false, UDFToString.class.getSimpleName(), null); + + system.registerGenericUDF(serdeConstants.DATE_TYPE_NAME, GenericUDFToDate.class, null); + system.registerGenericUDF(serdeConstants.TIMESTAMP_TYPE_NAME, GenericUDFTimestamp.class, null); + system.registerGenericUDF(serdeConstants.BINARY_TYPE_NAME, GenericUDFToBinary.class, null); + system.registerGenericUDF(serdeConstants.DECIMAL_TYPE_NAME, GenericUDFToDecimal.class, null); + system.registerGenericUDF(serdeConstants.VARCHAR_TYPE_NAME, GenericUDFToVarchar.class, null); + system.registerGenericUDF(serdeConstants.CHAR_TYPE_NAME, GenericUDFToChar.class, null); // Aggregate functions - registerGenericUDAF("max", new GenericUDAFMax()); - registerGenericUDAF("min", new GenericUDAFMin()); + system.registerGenericUDAF("max", new GenericUDAFMax(), null); + system.registerGenericUDAF("min", new GenericUDAFMin(), null); - registerGenericUDAF("sum", new GenericUDAFSum()); - registerGenericUDAF("count", new GenericUDAFCount()); - registerGenericUDAF("avg", new GenericUDAFAverage()); - registerGenericUDAF("std", new GenericUDAFStd()); - registerGenericUDAF("stddev", new GenericUDAFStd()); - registerGenericUDAF("stddev_pop", new GenericUDAFStd()); - registerGenericUDAF("stddev_samp", new GenericUDAFStdSample()); - registerGenericUDAF("variance", new GenericUDAFVariance()); - registerGenericUDAF("var_pop", new GenericUDAFVariance()); - registerGenericUDAF("var_samp", new GenericUDAFVarianceSample()); - registerGenericUDAF("covar_pop", new GenericUDAFCovariance()); - registerGenericUDAF("covar_samp", new GenericUDAFCovarianceSample()); - registerGenericUDAF("corr", new GenericUDAFCorrelation()); - registerGenericUDAF("histogram_numeric", new GenericUDAFHistogramNumeric()); - registerGenericUDAF("percentile_approx", new GenericUDAFPercentileApprox()); - registerGenericUDAF("collect_set", new GenericUDAFCollectSet()); - registerGenericUDAF("collect_list", new GenericUDAFCollectList()); + system.registerGenericUDAF("sum", new GenericUDAFSum(), null); + system.registerGenericUDAF("count", new GenericUDAFCount(), null); + system.registerGenericUDAF("avg", new GenericUDAFAverage(), null); + system.registerGenericUDAF("std", new GenericUDAFStd(), null); + system.registerGenericUDAF("stddev", new GenericUDAFStd(), null); + system.registerGenericUDAF("stddev_pop", new GenericUDAFStd(), null); + system.registerGenericUDAF("stddev_samp", new GenericUDAFStdSample(), null); + system.registerGenericUDAF("variance", new GenericUDAFVariance(), null); + system.registerGenericUDAF("var_pop", new GenericUDAFVariance(), null); + system.registerGenericUDAF("var_samp", new GenericUDAFVarianceSample(), null); + system.registerGenericUDAF("covar_pop", new GenericUDAFCovariance(), null); + system.registerGenericUDAF("covar_samp", new GenericUDAFCovarianceSample(), null); + system.registerGenericUDAF("corr", new GenericUDAFCorrelation(), null); + system.registerGenericUDAF("histogram_numeric", new GenericUDAFHistogramNumeric(), null); + system.registerGenericUDAF("percentile_approx", new GenericUDAFPercentileApprox(), null); + system.registerGenericUDAF("collect_set", new GenericUDAFCollectSet(), null); + system.registerGenericUDAF("collect_list", new GenericUDAFCollectList(), null); - registerGenericUDAF("ngrams", new GenericUDAFnGrams()); - registerGenericUDAF("context_ngrams", new GenericUDAFContextNGrams()); + system.registerGenericUDAF("ngrams", new GenericUDAFnGrams(), null); + system.registerGenericUDAF("context_ngrams", new GenericUDAFContextNGrams(), null); - registerGenericUDAF("ewah_bitmap", new GenericUDAFEWAHBitmap()); + system.registerGenericUDAF("ewah_bitmap", new GenericUDAFEWAHBitmap(), null); - registerGenericUDAF("compute_stats" , new GenericUDAFComputeStats()); + system.registerGenericUDAF("compute_stats", new GenericUDAFComputeStats(), null); - registerUDAF("percentile", UDAFPercentile.class); + system.registerUDAF("percentile", UDAFPercentile.class, null); // Generic UDFs - registerGenericUDF("reflect", GenericUDFReflect.class); - registerGenericUDF("reflect2", GenericUDFReflect2.class); - registerGenericUDF("java_method", GenericUDFReflect.class); - - registerGenericUDF("array", GenericUDFArray.class); - registerGenericUDF("assert_true", GenericUDFAssertTrue.class); - registerGenericUDF("map", GenericUDFMap.class); - registerGenericUDF("struct", GenericUDFStruct.class); - registerGenericUDF("named_struct", GenericUDFNamedStruct.class); - registerGenericUDF("create_union", GenericUDFUnion.class); - - registerGenericUDF("case", GenericUDFCase.class); - registerGenericUDF("when", GenericUDFWhen.class); - registerGenericUDF("hash", GenericUDFHash.class); - registerGenericUDF("coalesce", GenericUDFCoalesce.class); - registerGenericUDF("index", GenericUDFIndex.class); - registerGenericUDF("in_file", GenericUDFInFile.class); - registerGenericUDF("instr", GenericUDFInstr.class); - registerGenericUDF("locate", GenericUDFLocate.class); - registerGenericUDF("elt", GenericUDFElt.class); - registerGenericUDF("concat_ws", GenericUDFConcatWS.class); - registerGenericUDF("sort_array", GenericUDFSortArray.class); - registerGenericUDF("array_contains", GenericUDFArrayContains.class); - registerGenericUDF("sentences", GenericUDFSentences.class); - registerGenericUDF("map_keys", GenericUDFMapKeys.class); - registerGenericUDF("map_values", GenericUDFMapValues.class); - registerGenericUDF("format_number", GenericUDFFormatNumber.class); - registerGenericUDF("printf", GenericUDFPrintf.class); - - registerGenericUDF("from_utc_timestamp", GenericUDFFromUtcTimestamp.class); - registerGenericUDF("to_utc_timestamp", GenericUDFToUtcTimestamp.class); - - registerGenericUDF("unix_timestamp", GenericUDFUnixTimeStamp.class); - registerGenericUDF("to_unix_timestamp", GenericUDFToUnixTimeStamp.class); + system.registerGenericUDF("reflect", GenericUDFReflect.class, null); + system.registerGenericUDF("reflect2", GenericUDFReflect2.class, null); + system.registerGenericUDF("java_method", GenericUDFReflect.class, null); + + system.registerGenericUDF("array", GenericUDFArray.class, null); + system.registerGenericUDF("assert_true", GenericUDFAssertTrue.class, null); + system.registerGenericUDF("map", GenericUDFMap.class, null); + system.registerGenericUDF("struct", GenericUDFStruct.class, null); + system.registerGenericUDF("named_struct", GenericUDFNamedStruct.class, null); + system.registerGenericUDF("create_union", GenericUDFUnion.class, null); + + system.registerGenericUDF("case", GenericUDFCase.class, null); + system.registerGenericUDF("when", GenericUDFWhen.class, null); + system.registerGenericUDF("hash", GenericUDFHash.class, null); + system.registerGenericUDF("coalesce", GenericUDFCoalesce.class, null); + system.registerGenericUDF("index", GenericUDFIndex.class, null); + system.registerGenericUDF("in_file", GenericUDFInFile.class, null); + system.registerGenericUDF("instr", GenericUDFInstr.class, null); + system.registerGenericUDF("locate", GenericUDFLocate.class, null); + system.registerGenericUDF("elt", GenericUDFElt.class, null); + system.registerGenericUDF("concat_ws", GenericUDFConcatWS.class, null); + system.registerGenericUDF("sort_array", GenericUDFSortArray.class, null); + system.registerGenericUDF("array_contains", GenericUDFArrayContains.class, null); + system.registerGenericUDF("sentences", GenericUDFSentences.class, null); + system.registerGenericUDF("map_keys", GenericUDFMapKeys.class, null); + system.registerGenericUDF("map_values", GenericUDFMapValues.class, null); + system.registerGenericUDF("format_number", GenericUDFFormatNumber.class, null); + system.registerGenericUDF("printf", GenericUDFPrintf.class, null); + + system.registerGenericUDF("from_utc_timestamp", GenericUDFFromUtcTimestamp.class, null); + system.registerGenericUDF("to_utc_timestamp", GenericUDFToUtcTimestamp.class, null); + + system.registerGenericUDF("unix_timestamp", GenericUDFUnixTimeStamp.class, null); + system.registerGenericUDF("to_unix_timestamp", GenericUDFToUnixTimeStamp.class, null); // Generic UDTF's - registerGenericUDTF("explode", GenericUDTFExplode.class); - registerGenericUDTF("inline", GenericUDTFInline.class); - registerGenericUDTF("json_tuple", GenericUDTFJSONTuple.class); - registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class); - registerGenericUDTF("posexplode", GenericUDTFPosExplode.class); - registerGenericUDTF("stack", GenericUDTFStack.class); + system.registerGenericUDTF("explode", GenericUDTFExplode.class, null); + system.registerGenericUDTF("inline", GenericUDTFInline.class, null); + system.registerGenericUDTF("json_tuple", GenericUDTFJSONTuple.class, null); + system.registerGenericUDTF("parse_url_tuple", GenericUDTFParseUrlTuple.class, null); + system.registerGenericUDTF("posexplode", GenericUDTFPosExplode.class, null); + system.registerGenericUDTF("stack", GenericUDTFStack.class, null); //PTF declarations - registerGenericUDF(LEAD_FUNC_NAME, GenericUDFLead.class); - registerGenericUDF(LAG_FUNC_NAME, GenericUDFLag.class); - - registerWindowFunction("row_number", new GenericUDAFRowNumber()); - registerWindowFunction("rank", new GenericUDAFRank()); - registerWindowFunction("dense_rank", new GenericUDAFDenseRank()); - registerWindowFunction("percent_rank", new GenericUDAFPercentRank()); - registerWindowFunction("cume_dist", new GenericUDAFCumeDist()); - registerWindowFunction("ntile", new GenericUDAFNTile()); - registerWindowFunction("first_value", new GenericUDAFFirstValue()); - registerWindowFunction("last_value", new GenericUDAFLastValue()); - registerWindowFunction(LEAD_FUNC_NAME, new GenericUDAFLead(), false); - registerWindowFunction(LAG_FUNC_NAME, new GenericUDAFLag(), false); - - registerTableFunction(NOOP_TABLE_FUNCTION, NoopResolver.class); - registerTableFunction(NOOP_MAP_TABLE_FUNCTION, NoopWithMapResolver.class); - registerTableFunction(NOOP_STREAMING_TABLE_FUNCTION, NoopStreamingResolver.class); - registerTableFunction(NOOP_STREAMING_MAP_TABLE_FUNCTION, NoopWithMapStreamingResolver.class); - registerTableFunction(WINDOWING_TABLE_FUNCTION, WindowingTableFunctionResolver.class); - registerTableFunction("matchpath", MatchPathResolver.class); - } - - public static void registerTemporaryUDF(String functionName, - Class UDFClass, boolean isOperator) { - registerUDF(false, functionName, UDFClass, isOperator); - } - - static void registerUDF(String functionName, Class UDFClass, - boolean isOperator) { - registerUDF(true, functionName, UDFClass, isOperator); - } - - public static void registerUDF(boolean isNative, String functionName, - Class UDFClass, boolean isOperator) { - registerUDF(isNative, functionName, UDFClass, isOperator, functionName - .toLowerCase()); - } - - public static void registerUDF(String functionName, - Class UDFClass, boolean isOperator, String displayName) { - registerUDF(true, functionName, UDFClass, isOperator, displayName); - } - - public static void registerUDF(boolean isNative, String functionName, - Class UDFClass, boolean isOperator, String displayName) { - if (UDF.class.isAssignableFrom(UDFClass)) { - FunctionInfo fI = new FunctionInfo(isNative, displayName, - new GenericUDFBridge(displayName, isOperator, UDFClass.getName())); - mFunctions.put(functionName.toLowerCase(), fI); - registerNativeStatus(fI); - } else { - throw new RuntimeException("Registering UDF Class " + UDFClass - + " which does not extend " + UDF.class); - } - } - - public static void registerTemporaryGenericUDF(String functionName, - Class genericUDFClass) { - registerGenericUDF(false, functionName, genericUDFClass); - } - - static void registerGenericUDF(String functionName, - Class genericUDFClass) { - registerGenericUDF(true, functionName, genericUDFClass); - } - - public static void registerGenericUDF(boolean isNative, String functionName, - Class genericUDFClass) { - if (GenericUDF.class.isAssignableFrom(genericUDFClass)) { - FunctionInfo fI = new FunctionInfo(isNative, functionName, - (GenericUDF) ReflectionUtils.newInstance(genericUDFClass, null)); - mFunctions.put(functionName.toLowerCase(), fI); - registerNativeStatus(fI); - } else { - throw new RuntimeException("Registering GenericUDF Class " - + genericUDFClass + " which does not extend " + GenericUDF.class); - } - } - - public static void registerTemporaryGenericUDTF(String functionName, - Class genericUDTFClass) { - registerGenericUDTF(false, functionName, genericUDTFClass); - } - - static void registerGenericUDTF(String functionName, - Class genericUDTFClass) { - registerGenericUDTF(true, functionName, genericUDTFClass); - } - - public static void registerGenericUDTF(boolean isNative, String functionName, - Class genericUDTFClass) { - if (GenericUDTF.class.isAssignableFrom(genericUDTFClass)) { - FunctionInfo fI = new FunctionInfo(isNative, functionName, - (GenericUDTF) ReflectionUtils.newInstance(genericUDTFClass, null)); - mFunctions.put(functionName.toLowerCase(), fI); - registerNativeStatus(fI); - } else { - throw new RuntimeException("Registering GenericUDTF Class " - + genericUDTFClass + " which does not extend " + GenericUDTF.class); - } - } - - private static FunctionInfo getFunctionInfoFromMetastore(String functionName) { - FunctionInfo ret = null; - + system.registerGenericUDF(LEAD_FUNC_NAME, GenericUDFLead.class, null); + system.registerGenericUDF(LAG_FUNC_NAME, GenericUDFLag.class, null); + + system.registerGenericUDAF("row_number", new GenericUDAFRowNumber(), null); + system.registerGenericUDAF("rank", new GenericUDAFRank(), null); + system.registerGenericUDAF("dense_rank", new GenericUDAFDenseRank(), null); + system.registerGenericUDAF("percent_rank", new GenericUDAFPercentRank(), null); + system.registerGenericUDAF("cume_dist", new GenericUDAFCumeDist(), null); + system.registerGenericUDAF("ntile", new GenericUDAFNTile(), null); + system.registerGenericUDAF("first_value", new GenericUDAFFirstValue(), null); + system.registerGenericUDAF("last_value", new GenericUDAFLastValue(), null); + system.registerWindowFunction(LEAD_FUNC_NAME, new GenericUDAFLead()); + system.registerWindowFunction(LAG_FUNC_NAME, new GenericUDAFLag()); + + registerTableFunction(NOOP_TABLE_FUNCTION, NoopResolver.class, null); + registerTableFunction(NOOP_MAP_TABLE_FUNCTION, NoopWithMapResolver.class, null); + registerTableFunction(NOOP_STREAMING_TABLE_FUNCTION, NoopStreamingResolver.class, null); + registerTableFunction(NOOP_STREAMING_MAP_TABLE_FUNCTION, NoopWithMapStreamingResolver.class, null); + registerTableFunction(WINDOWING_TABLE_FUNCTION, WindowingTableFunctionResolver.class, null); + registerTableFunction(MATCH_PATH_TABLE_FUNCTION, MatchPathResolver.class, null); + } + + // register all permanent functions. need improvement + static { try { - String dbName; - String fName; - if (FunctionUtils.isQualifiedFunctionName(functionName)) { - String[] parts = FunctionUtils.splitQualifiedFunctionName(functionName); - dbName = parts[0]; - fName = parts[1]; - } else { - // otherwise, qualify using current db - dbName = SessionState.get().getCurrentDatabase().toLowerCase(); - fName = functionName; - } - - // Try looking up function in the metastore - HiveConf conf = SessionState.get().getConf(); - Function func = Hive.get(conf).getFunction(dbName, fName); - if (func != null) { - // Found UDF in metastore - now add it to the function registry - // At this point we should add any relevant jars that would be needed for the UDf. - try { - FunctionTask.addFunctionResources(func.getResourceUris()); - } catch (Exception e) { - LOG.error("Unable to load resources for " + dbName + "." + fName + ":" + e.getMessage(), e); - return null; - } - - Class udfClass = Class.forName(func.getClassName(), true, Utilities.getSessionSpecifiedClassLoader()); - if (registerTemporaryFunction(functionName, udfClass)) { - ret = mFunctions.get(functionName); - } else { - LOG.error(func.getClassName() + " is not a valid UDF class and was not registered."); + Hive db = Hive.get(); + for (String dbName : db.getAllDatabases()) { + for (String functionName : db.getFunctions(dbName, "*")) { + Function function = db.getFunction(dbName, functionName); + registerPermanentFunction(functionName, function.getClassName(), function.getResourceUris()); } } - } catch (HiveException e) { - if (!((e.getCause() != null) && (e.getCause() instanceof MetaException)) && - (e.getCause().getCause() != null) && (e.getCause().getCause() instanceof NoSuchObjectException)) { - LOG.info("Unable to lookup UDF in metastore: " + e); - } - } catch (ClassNotFoundException e) { - // Lookup of UDf class failed - LOG.error("Unable to load UDF class: " + e); - } - - return ret; - } - - private static T getQualifiedFunctionInfo( - Map mFunctions, String functionName) { - T functionInfo = mFunctions.get(functionName); - if (functionInfo == null) { - // Try looking up in metastore. - FunctionInfo fi = getFunctionInfoFromMetastore(functionName); - if (fi != null) { - // metastore lookup resulted in function getting added to mFunctions, try again - functionInfo = mFunctions.get(functionName); - } - } - - // HIVE-6672: In HiveServer2 the JARs for this UDF may have been loaded by a different thread, - // and the current thread may not be able to resolve the UDF. Test for this condition - // and if necessary load the JARs in this thread. - if (functionInfo != null) { - loadFunctionResourcesIfNecessary(functionName, functionInfo); + } catch (Exception e) { + throw new RuntimeException(e); } - - return functionInfo; } - private static void checkFunctionClass(CommonFunctionInfo cfi) throws ClassNotFoundException { - // This call will fail for non-generic UDFs using GenericUDFBridge - Class udfClass = cfi.getFunctionClass(); - // Even if we have a reference to the class (which will be the case for GenericUDFs), - // the classloader may not be able to resolve the class, which would mean reflection-based - // methods would fail such as for plan deserialization. Make sure this works too. - Class.forName(udfClass.getName(), true, Utilities.getSessionSpecifiedClassLoader()); - } - - private static void loadFunctionResourcesIfNecessary(String functionName, CommonFunctionInfo cfi) { - try { - // Check if the necessary JARs have been loaded for this function. - checkFunctionClass(cfi); - } catch (Exception e) { - // Unable to resolve the UDF with the classloader. - // Look up the function in the metastore and load any resources. - LOG.debug("Attempting to reload resources for " + functionName); - try { - String[] parts = FunctionUtils.getQualifiedFunctionNameParts(functionName); - HiveConf conf = SessionState.get().getConf(); - Function func = Hive.get(conf).getFunction(parts[0], parts[1]); - if (func != null) { - FunctionTask.addFunctionResources(func.getResourceUris()); - // Check again now that we've loaded the resources in this thread. - checkFunctionClass(cfi); - } else { - // Couldn't find the function .. just rethrow the original error - LOG.error("Unable to reload resources for " + functionName); - throw e; - } - } catch (Exception err) { - throw new RuntimeException(err); - } - } + public static FunctionInfo registerTableFunction(String function, + Class clazz, List resources) { + FunctionInfo info = new FunctionInfo(function, clazz, resources); + Registry registry = info.isNative()? system : SessionState.getRegistryForWrite(); + registry.addFunction(function, info); + return info; } public static String getNormalizedFunctionName(String fn) { // Does the same thing as getFunctionInfo, except for getting the function info. fn = fn.toLowerCase(); - return (FunctionUtils.isQualifiedFunctionName(fn) || mFunctions.get(fn) != null) ? fn + return (FunctionUtils.isQualifiedFunctionName(fn) || getFunctionInfo(fn) != null) ? fn : FunctionUtils.qualifyFunctionName( fn, SessionState.get().getCurrentDatabase().toLowerCase()); } - private static T getFunctionInfo( - Map mFunctions, String functionName) { - functionName = functionName.toLowerCase(); - T functionInfo = null; - if (FunctionUtils.isQualifiedFunctionName(functionName)) { - functionInfo = getQualifiedFunctionInfo(mFunctions, functionName); - } else { - // First try without qualifiers - would resolve builtin/temp functions. - // Otherwise try qualifying with current db name. - functionInfo = mFunctions.get(functionName); - if (functionInfo == null && !FunctionUtils.isQualifiedFunctionName(functionName)) { - String qualifiedName = FunctionUtils.qualifyFunctionName(functionName, - SessionState.get().getCurrentDatabase().toLowerCase()); - functionInfo = getQualifiedFunctionInfo(mFunctions, qualifiedName); - } - } - return functionInfo; - } - public static FunctionInfo getFunctionInfo(String functionName) { - return getFunctionInfo(mFunctions, functionName); + FunctionInfo info = getTemporaryFunctionInfo(functionName); + return info != null ? info : system.getFunctionInfo(functionName); } - /** - * Returns a set of registered function names. This is used for the CLI - * command "SHOW FUNCTIONS;" - * - * @return set of strings contains function names - */ - public static Set getFunctionNames() { - return getFunctionNames(true); + public static FunctionInfo getTemporaryFunctionInfo(String functionName) { + Registry registry = SessionState.getRegistry(); + return registry == null ? null : registry.getFunctionInfo(functionName); } - private static Set getFunctionNames(boolean searchMetastore) { - Set functionNames = mFunctions.keySet(); - if (searchMetastore) { - functionNames = new HashSet(functionNames); - try { - Hive db = getHive(); - List dbNames = db.getAllDatabases(); - - for (String dbName : dbNames) { - List funcNames = db.getFunctions(dbName, "*"); - for (String funcName : funcNames) { - functionNames.add(FunctionUtils.qualifyFunctionName(funcName, dbName)); - } - } - } catch (Exception e) { - LOG.error(e); - // Continue on, we can still return the functions we've gotten to this point. - } - } - return functionNames; + public static WindowFunctionInfo getWindowFunctionInfo(String functionName) { + Registry registry = SessionState.getRegistry(); + WindowFunctionInfo info = registry == null ? null : registry.getWindowFunctionInfo(functionName); + return info != null ? info : system.getWindowFunctionInfo(functionName); } - public static Hive getHive() throws HiveException { - return Hive.get(SessionState.get().getConf()); + public static Set getFunctionNames() { + Set names = new TreeSet(); + names.addAll(system.getCurrentFunctionNames()); + if (SessionState.getRegistry() != null) { + names.addAll(SessionState.getRegistry().getCurrentFunctionNames()); + } + return names; } - /** - * Returns a set of registered function names. This is used for the CLI - * command "SHOW FUNCTIONS 'regular expression';" Returns an empty set when - * the regular expression is not valid. - * - * @param funcPatternStr - * regular expression of the interested function names - * @return set of strings contains function names - */ public static Set getFunctionNames(String funcPatternStr) { Set funcNames = new TreeSet(); - Pattern funcPattern = null; - try { - funcPattern = Pattern.compile(funcPatternStr); - } catch (PatternSyntaxException e) { - return funcNames; - } - for (String funcName : mFunctions.keySet()) { - if (funcPattern.matcher(funcName).matches()) { - funcNames.add(funcName); - } + funcNames.addAll(system.getFunctionNames(funcPatternStr)); + if (SessionState.getRegistry() != null) { + funcNames.addAll(SessionState.getRegistry().getFunctionNames(funcPatternStr)); } return funcNames; } @@ -741,7 +493,7 @@ public static Hive getHive() throws HiveException { */ public static Set getFunctionNamesByLikePattern(String funcPatternStr) { Set funcNames = new TreeSet(); - Set allFuncs = getFunctionNames(true); + Set allFuncs = getFunctionNames(); String[] subpatterns = funcPatternStr.trim().split("\\|"); for (String subpattern : subpatterns) { subpattern = "(?i)" + subpattern.replaceAll("\\*", ".*"); @@ -762,28 +514,19 @@ public static Hive getHive() throws HiveException { /** * Returns the set of synonyms of the supplied function. * - * @param funcName - * the name of the function + * @param funcName the name of the function * @return Set of synonyms for funcName */ public static Set getFunctionSynonyms(String funcName) { - Set synonyms = new HashSet(); - FunctionInfo funcInfo = getFunctionInfo(funcName); if (null == funcInfo) { - return synonyms; + return Collections.emptySet(); } - - Class funcClass = funcInfo.getFunctionClass(); - for (String name : mFunctions.keySet()) { - if (name.equals(funcName)) { - continue; - } - if (mFunctions.get(name).getFunctionClass().equals(funcClass)) { - synonyms.add(name); - } + Set synonyms = new HashSet(); + system.getFunctionSynonyms(funcName, funcInfo, synonyms); + if (SessionState.getRegistry() != null) { + SessionState.getRegistry().getFunctionSynonyms(funcName, funcInfo, synonyms); } - return synonyms; } @@ -1018,7 +761,7 @@ public static PrimitiveCategory getCommonCategory(TypeInfo a, TypeInfo b) { // If either is not a numeric type, return null. return null; } - + return (ai > bi) ? pcA : pcB; } @@ -1152,20 +895,11 @@ public static GenericUDAFEvaluator getGenericUDAFEvaluator(String name, public static GenericUDAFEvaluator getGenericWindowingEvaluator(String name, List argumentOIs, boolean isDistinct, boolean isAllColumns) throws SemanticException { - - WindowFunctionInfo finfo = windowFunctions.get(name.toLowerCase()); - if (finfo == null) { return null;} - if ( !name.toLowerCase().equals(LEAD_FUNC_NAME) && - !name.toLowerCase().equals(LAG_FUNC_NAME) ) { - return getGenericUDAFEvaluator(name, argumentOIs, isDistinct, isAllColumns); - } - - // this must be lead/lag UDAF - ObjectInspector args[] = new ObjectInspector[argumentOIs.size()]; - GenericUDAFResolver udafResolver = finfo.getfInfo().getGenericUDAFResolver(); - GenericUDAFParameterInfo paramInfo = new SimpleGenericUDAFParameterInfo( - argumentOIs.toArray(args), isDistinct, isAllColumns); - return ((GenericUDAFResolver2) udafResolver).getEvaluator(paramInfo); + Registry registry = SessionState.getRegistry(); + GenericUDAFEvaluator evaluator = registry == null ? null : + registry.getGenericWindowingEvaluator(name, argumentOIs, isDistinct, isAllColumns); + return evaluator != null ? evaluator : + system.getGenericWindowingEvaluator(name, argumentOIs, isDistinct, isAllColumns); } /** @@ -1189,61 +923,6 @@ public static GenericUDAFEvaluator getGenericWindowingEvaluator(String name, return getMethodInternal(udfClass, mlist, exact, argumentClasses); } - public static void registerTemporaryGenericUDAF(String functionName, - GenericUDAFResolver genericUDAFResolver) { - registerGenericUDAF(false, functionName, genericUDAFResolver); - } - - static void registerGenericUDAF(String functionName, - GenericUDAFResolver genericUDAFResolver) { - registerGenericUDAF(true, functionName, genericUDAFResolver); - } - - public static void registerGenericUDAF(boolean isNative, String functionName, - GenericUDAFResolver genericUDAFResolver) { - FunctionInfo fi = new FunctionInfo(isNative, functionName.toLowerCase(), genericUDAFResolver); - mFunctions.put(functionName.toLowerCase(), fi); - - // All aggregate functions should also be usable as window functions - addFunctionInfoToWindowFunctions(functionName, fi); - - registerNativeStatus(fi); - } - - public static void registerTemporaryUDAF(String functionName, - Class udafClass) { - registerUDAF(false, functionName, udafClass); - } - - static void registerUDAF(String functionName, Class udafClass) { - registerUDAF(true, functionName, udafClass); - } - - public static void registerUDAF(boolean isNative, String functionName, - Class udafClass) { - FunctionInfo fi = new FunctionInfo(isNative, - functionName.toLowerCase(), new GenericUDAFBridge( - (UDAF) ReflectionUtils.newInstance(udafClass, null))); - mFunctions.put(functionName.toLowerCase(), fi); - - // All aggregate functions should also be usable as window functions - addFunctionInfoToWindowFunctions(functionName, fi); - - registerNativeStatus(fi); - } - - public static void unregisterTemporaryUDF(String functionName) throws HiveException { - FunctionInfo fi = mFunctions.get(functionName.toLowerCase()); - if (fi != null) { - if (!fi.isNative()) { - mFunctions.remove(functionName.toLowerCase()); - } else { - throw new HiveException("Function " + functionName - + " is hive native, it can't be dropped"); - } - } - } - public static GenericUDAFResolver getGenericUDAFResolver(String functionName) { if (LOG.isDebugEnabled()) { LOG.debug("Looking up GenericUDAF: " + functionName); @@ -1571,8 +1250,7 @@ public static GenericUDF cloneGenericUDF(GenericUDF genericUDF) { clonedUDF = new GenericUDFMacro(bridge.getMacroName(), bridge.getBody(), bridge.getColNames(), bridge.getColTypes()); } else { - clonedUDF = (GenericUDF) ReflectionUtils - .newInstance(genericUDF.getClass(), null); + clonedUDF = ReflectionUtils.newInstance(genericUDF.getClass(), null); } if (clonedUDF != null) { @@ -1610,32 +1288,23 @@ public static GenericUDTF cloneGenericUDTF(GenericUDTF genericUDTF) { if (null == genericUDTF) { return null; } - return (GenericUDTF) ReflectionUtils.newInstance(genericUDTF.getClass(), - null); + return ReflectionUtils.newInstance(genericUDTF.getClass(), null); } /** * Get the UDF class from an exprNodeDesc. Returns null if the exprNodeDesc * does not contain a UDF class. */ - private static Class getGenericUDFClassFromExprDesc(ExprNodeDesc desc) { + private static Class getUDFClassFromExprDesc(ExprNodeDesc desc) { if (!(desc instanceof ExprNodeGenericFuncDesc)) { return null; } ExprNodeGenericFuncDesc genericFuncDesc = (ExprNodeGenericFuncDesc) desc; - return genericFuncDesc.getGenericUDF().getClass(); - } - - /** - * Get the UDF class from an exprNodeDesc. Returns null if the exprNodeDesc - * does not contain a UDF class. - */ - private static Class getUDFClassFromExprDesc(ExprNodeDesc desc) { - if (!(desc instanceof ExprNodeGenericFuncDesc)) { - return null; + GenericUDF genericUDF = genericFuncDesc.getGenericUDF(); + if (genericUDF instanceof GenericUDFBridge) { + return ((GenericUDFBridge) genericUDF).getUdfClass(); } - ExprNodeGenericFuncDesc genericFuncDesc = (ExprNodeGenericFuncDesc) desc; - return genericFuncDesc.getGenericUDF().getClass(); + return genericUDF.getClass(); } /** @@ -1697,7 +1366,7 @@ public static boolean isStateful(GenericUDF genericUDF) { * Returns whether the exprNodeDesc is a node of "and", "or", "not". */ public static boolean isOpAndOrNot(ExprNodeDesc desc) { - Class genericUdfClass = getGenericUDFClassFromExprDesc(desc); + Class genericUdfClass = getUDFClassFromExprDesc(desc); return GenericUDFOPAnd.class == genericUdfClass || GenericUDFOPOr.class == genericUdfClass || GenericUDFOPNot.class == genericUdfClass; @@ -1707,21 +1376,21 @@ public static boolean isOpAndOrNot(ExprNodeDesc desc) { * Returns whether the exprNodeDesc is a node of "and". */ public static boolean isOpAnd(ExprNodeDesc desc) { - return GenericUDFOPAnd.class == getGenericUDFClassFromExprDesc(desc); + return GenericUDFOPAnd.class == getUDFClassFromExprDesc(desc); } /** * Returns whether the exprNodeDesc is a node of "or". */ public static boolean isOpOr(ExprNodeDesc desc) { - return GenericUDFOPOr.class == getGenericUDFClassFromExprDesc(desc); + return GenericUDFOPOr.class == getUDFClassFromExprDesc(desc); } /** * Returns whether the exprNodeDesc is a node of "not". */ public static boolean isOpNot(ExprNodeDesc desc) { - return GenericUDFOPNot.class == getGenericUDFClassFromExprDesc(desc); + return GenericUDFOPNot.class == getUDFClassFromExprDesc(desc); } /** @@ -1762,6 +1431,10 @@ public static boolean isOpPreserveInputName(ExprNodeDesc desc) { return isOpCast(desc); } + public static FunctionInfo registerTemporaryUDF(String functionName, Class udfClass) { + return registerTemporaryUDF(functionName, udfClass, null); + } + /** * Registers the appropriate kind of temporary function based on a class's * type. @@ -1773,103 +1446,41 @@ public static boolean isOpPreserveInputName(ExprNodeDesc desc) { * @return true if udfClass's type was recognized (so registration * succeeded); false otherwise */ - public static boolean registerTemporaryFunction( - String functionName, Class udfClass) { - - UDFClassType udfClassType = FunctionUtils.getUDFClassType(udfClass); - switch (udfClassType) { - case UDF: - FunctionRegistry.registerTemporaryUDF( - functionName, (Class) udfClass, false); - break; - case GENERIC_UDF: - FunctionRegistry.registerTemporaryGenericUDF( - functionName, (Class) udfClass); - break; - case GENERIC_UDTF: - FunctionRegistry.registerTemporaryGenericUDTF( - functionName, (Class) udfClass); - break; - case UDAF: - FunctionRegistry.registerTemporaryUDAF( - functionName, (Class) udfClass); - break; - case GENERIC_UDAF_RESOLVER: - FunctionRegistry.registerTemporaryGenericUDAF( - functionName, (GenericUDAFResolver) - ReflectionUtils.newInstance(udfClass, null)); - break; - case TABLE_FUNCTION_RESOLVER: - FunctionRegistry.registerTableFunction( - functionName, (Class)udfClass); - break; - default: - return false; - } - return true; + public static FunctionInfo registerTemporaryUDF( + String functionName, Class udfClass, List resources) { + return SessionState.getRegistryForWrite().registerFunction( + functionName, udfClass, resources); + } + public static void unregisterTemporaryUDF(String functionName) throws HiveException { + if (SessionState.getRegistry() != null) { + SessionState.getRegistry().unregisterFunction(functionName); + } } /** - * Registers thae appropriate kind of temporary function based on a class's + * Registers the appropriate kind of temporary function based on a class's * type. * * @param macroName name under which to register the macro * * @param body the expression which the macro evaluates to - * - * @param colNames the names of the arguments to the macro - * - * @param colTypes the types of the arguments to the macro + *@param colNames the names of the arguments to the macro + *@param colTypes the types of the arguments to the macro */ public static void registerTemporaryMacro( - String macroName, ExprNodeDesc body, - List colNames, List colTypes) { - - FunctionInfo fI = new FunctionInfo(false, macroName, - new GenericUDFMacro(macroName, body, colNames, colTypes)); - mFunctions.put(macroName.toLowerCase(), fI); - registerNativeStatus(fI); + String macroName, ExprNodeDesc body, List colNames, List colTypes) { + SessionState.getRegistryForWrite().registerMacro(macroName, body, colNames, colTypes); } - /** - * Registers Hive functions from a plugin jar, using metadata from - * the jar's META-INF/class-info.xml. - * - * @param jarLocation URL for reading jar file - * - * @param classLoader classloader to use for loading function classes - */ - public static void registerFunctionsFromPluginJar( - URL jarLocation, - ClassLoader classLoader) throws Exception { + public static FunctionInfo registerPermanentFunction(String functionName, + String className, List resources) { + return system.registerPermanentFunction(functionName, className, resources); + } - URL url = new URL("jar:" + jarLocation + "!/META-INF/class-info.xml"); - InputStream inputStream = null; - try { - inputStream = url.openStream(); - DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance(); - DocumentBuilder docBuilder = dbf.newDocumentBuilder(); - Document doc = docBuilder.parse(inputStream); - Element root = doc.getDocumentElement(); - if (!root.getTagName().equals("ClassList")) { - return; - } - NodeList children = root.getElementsByTagName("Class"); - for (int i = 0; i < children.getLength(); ++i) { - Element child = (Element) children.item(i); - String javaName = child.getAttribute("javaname"); - String sqlName = child.getAttribute("sqlname"); - Class udfClass = Class.forName(javaName, true, classLoader); - boolean registered = registerTemporaryFunction(sqlName, udfClass); - if (!registered) { - throw new RuntimeException( - "Class " + udfClass + " is not a Hive function implementation"); - } - } - } finally { - IOUtils.closeStream(inputStream); - } + public static void unregisterPermanentFunction(String functionName) throws HiveException { + system.unregisterFunction(functionName); + unregisterTemporaryUDF(functionName); } private FunctionRegistry() { @@ -1879,42 +1490,6 @@ private FunctionRegistry() { //---------PTF functions------------ - public static void registerWindowFunction(String name, GenericUDAFResolver wFn) - { - registerWindowFunction(name, wFn, true); - } - - /** - * Typically a WindowFunction is the same as a UDAF. The only exceptions are Lead & Lag UDAFs. These - * are not registered as regular UDAFs because - * - we plan to support Lead & Lag as UDFs (usable only within argument expressions - * of UDAFs when windowing is involved). Since mFunctions holds both UDFs and UDAFs we cannot - * add both FunctionInfos to mFunctions. - * We choose to only register UDFs in mFunctions. The implication of this is that Lead/Lag UDAFs - * are only usable when windowing is involved. - * - * @param name - * @param wFn - * @param registerAsUDAF - */ - public static void registerWindowFunction(String name, GenericUDAFResolver wFn, boolean registerAsUDAF) - { - FunctionInfo fInfo = null; - if (registerAsUDAF) { - // Just register the function normally, will also get added to window functions. - registerGenericUDAF(true, name, wFn); - } - else { - name = name.toLowerCase(); - fInfo = new FunctionInfo(true, name, wFn); - addFunctionInfoToWindowFunctions(name, fInfo); - } - } - - public static WindowFunctionInfo getWindowFunctionInfo(String functionName) { - return getFunctionInfo(windowFunctions, functionName); - } - /** * Both UDF and UDAF functions can imply order for analytical functions * @@ -1926,13 +1501,11 @@ public static WindowFunctionInfo getWindowFunctionInfo(String functionName) { public static boolean impliesOrder(String functionName) { FunctionInfo info = getFunctionInfo(functionName); - if (info != null) { - if (info.isGenericUDF()) { - UDFType type = - AnnotationUtils.getAnnotation(info.getGenericUDF().getClass(), UDFType.class); - if (type != null) { - return type.impliesOrder(); - } + if (info != null && info.isGenericUDF()) { + UDFType type = + AnnotationUtils.getAnnotation(info.getGenericUDF().getClass(), UDFType.class); + if (type != null) { + return type.impliesOrder(); } } WindowFunctionInfo windowInfo = getWindowFunctionInfo(functionName); @@ -1942,33 +1515,23 @@ public static boolean impliesOrder(String functionName) { return false; } - static private void addFunctionInfoToWindowFunctions(String functionName, - FunctionInfo functionInfo) { - // Assumes that the caller has already verified that functionInfo is for an aggregate function - WindowFunctionInfo wInfo = new WindowFunctionInfo(functionInfo); - windowFunctions.put(functionName.toLowerCase(), wInfo); - } - - public static boolean isTableFunction(String name) - { - FunctionInfo tFInfo = getFunctionInfo(name); + public static boolean isTableFunction(String functionName) { + FunctionInfo tFInfo = getFunctionInfo(functionName); return tFInfo != null && !tFInfo.isInternalTableFunction() && tFInfo.isTableFunction(); } - public static TableFunctionResolver getTableFunctionResolver(String name) - { - FunctionInfo tfInfo = getFunctionInfo(name); - if(tfInfo.isTableFunction()) { + public static TableFunctionResolver getTableFunctionResolver(String functionName) { + FunctionInfo tfInfo = getFunctionInfo(functionName); + if (tfInfo != null && tfInfo.isTableFunction()) { return (TableFunctionResolver) ReflectionUtils.newInstance(tfInfo.getFunctionClass(), null); } return null; } - public static TableFunctionResolver getWindowingTableFunction() - { + public static TableFunctionResolver getWindowingTableFunction() { return getTableFunctionResolver(WINDOWING_TABLE_FUNCTION); } - + public static boolean isNoopFunction(String fnName) { fnName = fnName.toLowerCase(); return fnName.equals(NOOP_MAP_TABLE_FUNCTION) || @@ -1977,13 +1540,6 @@ public static boolean isNoopFunction(String fnName) { fnName.equals(NOOP_STREAMING_TABLE_FUNCTION); } - public static void registerTableFunction(String name, Class tFnCls) - { - FunctionInfo tInfo = new FunctionInfo(name, tFnCls); - mFunctions.put(name.toLowerCase(), tInfo); - registerNativeStatus(tInfo); - } - /** * Use this to check if function is ranking function * @@ -1992,10 +1548,10 @@ public static void registerTableFunction(String name, Class udfClass = getUDFClassFromExprDesc(fnExpr); - if (udfClass == null) { - udfClass = getGenericUDFClassFromExprDesc(fnExpr); + public static boolean isBuiltInFuncExpr(ExprNodeGenericFuncDesc fnExpr) { + Class udfClass = FunctionRegistry.getUDFClassFromExprDesc(fnExpr); + if (udfClass != null) { + return system.isBuiltInFunc(udfClass); } - return nativeUdfs.contains(udfClass); - } - - private static void registerNativeStatus(FunctionInfo fi) { - if (!fi.isNative()) { - return; - } - nativeUdfs.add(fi.getFunctionClass()); + return false; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java index 569c125..460f077 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java @@ -26,7 +26,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.PrincipalType; @@ -34,7 +33,6 @@ import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.exec.FunctionUtils.UDFClassType; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.CreateFunctionDesc; @@ -110,6 +108,7 @@ public int execute(DriverContext driverContext) { return 0; } + // todo authorization private int createPermanentFunction(Hive db, CreateFunctionDesc createFunctionDesc) throws HiveException, IOException { String[] qualifiedNameParts = FunctionUtils.getQualifiedFunctionNameParts( @@ -118,64 +117,43 @@ private int createPermanentFunction(Hive db, CreateFunctionDesc createFunctionDe String funcName = qualifiedNameParts[1]; String registeredName = FunctionUtils.qualifyFunctionName(funcName, dbName); String className = createFunctionDesc.getClassName(); - boolean addedToRegistry = false; - try { - // For permanent functions, check for any resources from local filesystem. - checkLocalFunctionResources(db, createFunctionDesc.getResources()); - - // Add any required resources - addFunctionResources(createFunctionDesc.getResources()); - - // UDF class should exist - Class udfClass = getUdfClass(createFunctionDesc); - if (FunctionUtils.getUDFClassType(udfClass) == UDFClassType.UNKNOWN) { - console.printError("FAILED: Class " + createFunctionDesc.getClassName() - + " does not implement UDF, GenericUDF, or UDAF"); - return 1; - } - - // TODO: There should be a registerPermanentFunction() - addedToRegistry = FunctionRegistry.registerTemporaryFunction(registeredName, udfClass); - if (!addedToRegistry) { - console.printError("Failed to register " + registeredName - + " using class " + createFunctionDesc.getClassName()); - return 1; - } + List resources = createFunctionDesc.getResources(); - // Add to metastore - Function func = new Function( - funcName, - dbName, - className, - SessionState.get().getUserName(), - PrincipalType.USER, - (int) (System.currentTimeMillis() / 1000), - org.apache.hadoop.hive.metastore.api.FunctionType.JAVA, - createFunctionDesc.getResources() - ); - db.createFunction(func); - return 0; - } catch (ClassNotFoundException e) { - console.printError("FAILED: Class " + createFunctionDesc.getClassName() + " not found"); - LOG.info("create function: " + StringUtils.stringifyException(e)); - if (addedToRegistry) { - FunctionRegistry.unregisterTemporaryUDF(registeredName); - } + FunctionInfo registered = + FunctionRegistry.registerPermanentFunction(registeredName, className, resources); + if (registered == null) { + console.printError("Failed to register " + registeredName + + " using class " + createFunctionDesc.getClassName()); return 1; } + + // Add to metastore + Function func = new Function( + funcName, + dbName, + className, + SessionState.get().getUserName(), + PrincipalType.USER, + (int) (System.currentTimeMillis() / 1000), + org.apache.hadoop.hive.metastore.api.FunctionType.JAVA, + resources + ); + db.createFunction(func); + return 0; } private int createTemporaryFunction(CreateFunctionDesc createFunctionDesc) { try { // Add any required resources - addFunctionResources(createFunctionDesc.getResources()); + List resources = createFunctionDesc.getResources(); + addFunctionResources(resources); Class udfClass = getUdfClass(createFunctionDesc); - boolean registered = FunctionRegistry.registerTemporaryFunction( - createFunctionDesc.getFunctionName(), - udfClass); - if (registered) { + FunctionInfo registered = FunctionRegistry.registerTemporaryUDF( + createFunctionDesc.getFunctionName(), + udfClass, resources); + if (registered != null) { return 0; } console.printError("FAILED: Class " + createFunctionDesc.getClassName() @@ -198,14 +176,14 @@ private int createMacro(CreateMacroDesc createMacroDesc) { createMacroDesc.getMacroName(), createMacroDesc.getBody(), createMacroDesc.getColNames(), - createMacroDesc.getColTypes()); + createMacroDesc.getColTypes() + ); return 0; } private int dropMacro(DropMacroDesc dropMacroDesc) { try { - FunctionRegistry.unregisterTemporaryUDF(dropMacroDesc - .getMacroName()); + FunctionRegistry.unregisterTemporaryUDF(dropMacroDesc.getMacroName()); return 0; } catch (HiveException e) { LOG.info("drop macro: " + StringUtils.stringifyException(e)); @@ -213,6 +191,7 @@ private int dropMacro(DropMacroDesc dropMacroDesc) { } } + // todo authorization private int dropPermanentFunction(Hive db, DropFunctionDesc dropFunctionDesc) { try { String[] qualifiedNameParts = FunctionUtils.getQualifiedFunctionNameParts( @@ -221,8 +200,7 @@ private int dropPermanentFunction(Hive db, DropFunctionDesc dropFunctionDesc) { String funcName = qualifiedNameParts[1]; String registeredName = FunctionUtils.qualifyFunctionName(funcName, dbName); - // TODO: there should be a unregisterPermanentUDF() - FunctionRegistry.unregisterTemporaryUDF(registeredName); + FunctionRegistry.unregisterPermanentFunction(registeredName); db.dropFunction(dbName, funcName); return 0; @@ -235,8 +213,7 @@ private int dropPermanentFunction(Hive db, DropFunctionDesc dropFunctionDesc) { private int dropTemporaryFunction(DropFunctionDesc dropFunctionDesc) { try { - FunctionRegistry.unregisterTemporaryUDF(dropFunctionDesc - .getFunctionName()); + FunctionRegistry.unregisterTemporaryUDF(dropFunctionDesc.getFunctionName()); return 0; } catch (HiveException e) { LOG.info("drop function: " + StringUtils.stringifyException(e)); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java new file mode 100644 index 0000000..cde13c6 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Registry.java @@ -0,0 +1,465 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.api.ResourceUri; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFBridge; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFParameterInfo; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; +import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; +import org.apache.hadoop.hive.ql.udf.ptf.TableFunctionResolver; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.util.ReflectionUtils; + +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +// Extracted from FunctionRegistry +public class Registry { + + private static final Log LOG = LogFactory.getLog(FunctionRegistry.class); + + // prefix to discern LEAD/LAG UDFs from window functions with the same name + private static final String WINDOW_FUNC_PREFIX = "@_"; + + /** + * The mapping from expression function names to expression classes. + */ + private final Map mFunctions = new LinkedHashMap(); + private final Set> builtIns = Collections.synchronizedSet(new HashSet>()); + + private final boolean isNative; + + Registry(boolean isNative) { + this.isNative = isNative; + } + + public Registry() { + this(false); + } + + /** + * Registers the appropriate kind of temporary function based on a class's + * type. + * + * @param functionName name under which to register function + * @param udfClass class implementing UD[A|T]F + * @return true if udfClass's type was recognized (so registration + * succeeded); false otherwise + */ + @SuppressWarnings("unchecked") + public FunctionInfo registerFunction( + String functionName, Class udfClass, List resources) { + + FunctionUtils.UDFClassType udfClassType = FunctionUtils.getUDFClassType(udfClass); + switch (udfClassType) { + case UDF: + return registerUDF( + functionName, (Class) udfClass, false, resources); + case GENERIC_UDF: + return registerGenericUDF( + functionName, (Class) udfClass, resources); + case GENERIC_UDTF: + return registerGenericUDTF( + functionName, (Class) udfClass, resources); + case UDAF: + return registerUDAF( + functionName, (Class) udfClass, resources); + case GENERIC_UDAF_RESOLVER: + return registerGenericUDAF( + functionName, (GenericUDAFResolver) + ReflectionUtils.newInstance(udfClass, null), resources); + case TABLE_FUNCTION_RESOLVER: + // native or not would be decided by annotation. need to evaluate that first + return FunctionRegistry.registerTableFunction(functionName, + (Class) udfClass, resources); + } + return null; + + } + + public FunctionInfo registerUDF(String functionName, + Class UDFClass, boolean isOperator, List resources) { + return registerUDF(functionName, UDFClass, isOperator, functionName.toLowerCase(), resources); + } + + public FunctionInfo registerUDF(String functionName, + Class UDFClass, boolean isOperator, String displayName, + List resources) { + validateClass(UDFClass, UDF.class); + FunctionInfo fI = new FunctionInfo(isNative, displayName, + new GenericUDFBridge(displayName, isOperator, UDFClass.getName()), resources); + addFunction(functionName, fI); + return fI; + } + + public FunctionInfo registerGenericUDF(String functionName, + Class genericUDFClass, List resources) { + validateClass(genericUDFClass, GenericUDF.class); + FunctionInfo fI = new FunctionInfo(isNative, functionName, + ReflectionUtils.newInstance(genericUDFClass, null), resources); + addFunction(functionName, fI); + return fI; + } + + public FunctionInfo registerGenericUDTF(String functionName, + Class genericUDTFClass, List resources) { + validateClass(genericUDTFClass, GenericUDTF.class); + FunctionInfo fI = new FunctionInfo(isNative, functionName, + ReflectionUtils.newInstance(genericUDTFClass, null), resources); + addFunction(functionName, fI); + return fI; + } + + public FunctionInfo registerGenericUDAF(String functionName, + GenericUDAFResolver genericUDAFResolver, List resources) { + FunctionInfo function = + new WindowFunctionInfo(isNative, functionName, genericUDAFResolver, resources); + addFunction(functionName, function); + addFunction(WINDOW_FUNC_PREFIX + functionName, function); + return function; + } + + public FunctionInfo registerUDAF(String functionName, + Class udafClass, List resources) { + validateClass(udafClass, UDAF.class); + FunctionInfo function = new WindowFunctionInfo(isNative, functionName, + new GenericUDAFBridge(ReflectionUtils.newInstance(udafClass, null)), resources); + addFunction(functionName, function); + addFunction(WINDOW_FUNC_PREFIX + functionName, function); + return function; + } + + public FunctionInfo registerMacro(String macroName, ExprNodeDesc body, + List colNames, List colTypes) { + return registerMacro(macroName, body, colNames, colTypes, null); + } + + public FunctionInfo registerMacro(String macroName, ExprNodeDesc body, + List colNames, List colTypes, List resources) { + GenericUDFMacro macro = new GenericUDFMacro(macroName, body, colNames, colTypes); + FunctionInfo fI = new FunctionInfo(isNative, macroName, macro, resources); + addFunction(macroName, fI); + return fI; + } + + public FunctionInfo registerPermanentFunction(String functionName, + String className, List resources) { + FunctionInfo function = new FunctionInfo(functionName, className, resources); + // register to session first for backward compatibility + String qualifiedName = FunctionUtils.qualifyFunctionName( + functionName, SessionState.get().getCurrentDatabase().toLowerCase()); + if (registerToSessionRegistry(qualifiedName, function) != null) { + addFunction(functionName, function); + return function; + } + return null; + } + + /** + * Typically a WindowFunction is the same as a UDAF. The only exceptions are Lead & Lag UDAFs. These + * are not registered as regular UDAFs because + * - we plan to support Lead & Lag as UDFs (usable only within argument expressions + * of UDAFs when windowing is involved). Since mFunctions holds both UDFs and UDAFs we cannot + * add both FunctionInfos to mFunctions. + * We choose to only register UDFs in mFunctions. The implication of this is that Lead/Lag UDAFs + * are only usable when windowing is involved. + * + * @param name + * @param wFn + * @param registerAsUDAF + */ + void registerWindowFunction(String name, GenericUDAFResolver wFn) { + addFunction(WINDOW_FUNC_PREFIX + name, new WindowFunctionInfo(isNative, name, wFn, null)); + } + + private void validateClass(Class input, Class expected) { + if (!expected.isAssignableFrom(input)) { + throw new RuntimeException("Registering UDF Class " + input + + " which does not extend " + expected); + } + } + + /** + * Looks up the function name in the registry. If enabled, will attempt to search the metastore + * for the function. + * @param functionName + * @return + */ + public synchronized FunctionInfo getFunctionInfo(String functionName) { + functionName = functionName.toLowerCase(); + if (FunctionUtils.isQualifiedFunctionName(functionName)) { + return getQualifiedFunctionInfo(functionName); + } + // First try without qualifiers - would resolve builtin/temp functions. + // Otherwise try qualifying with current db name. + FunctionInfo functionInfo = mFunctions.get(functionName); + if (functionInfo == null) { + String qualifiedName = FunctionUtils.qualifyFunctionName( + functionName, SessionState.get().getCurrentDatabase().toLowerCase()); + functionInfo = getQualifiedFunctionInfo(qualifiedName); + } + return functionInfo; + } + + public WindowFunctionInfo getWindowFunctionInfo(String functionName) { + FunctionInfo info = getFunctionInfo(WINDOW_FUNC_PREFIX + functionName); + if (info instanceof WindowFunctionInfo) { + return (WindowFunctionInfo) info; + } + return null; + } + + /** + * @param fnExpr Function expression. + * @return True iff the fnExpr represents a hive built-in function. + */ + public boolean isBuiltInFunc(Class udfClass) { + return udfClass != null && builtIns.contains(udfClass); + } + + public synchronized Set getCurrentFunctionNames() { + return getFunctionNames((Pattern)null); + } + + public synchronized Set getFunctionNames(String funcPatternStr) { + try { + return getFunctionNames(Pattern.compile(funcPatternStr)); + } catch (PatternSyntaxException e) { + return Collections.emptySet(); + } + } + + /** + * Returns a set of registered function names. This is used for the CLI + * command "SHOW FUNCTIONS 'regular expression';" Returns an empty set when + * the regular expression is not valid. + * + * @param funcPatternStr regular expression of the interested function names + * @return set of strings contains function names + */ + public synchronized Set getFunctionNames(Pattern funcPattern) { + Set funcNames = new TreeSet(); + for (String funcName : mFunctions.keySet()) { + if (funcName.contains(WINDOW_FUNC_PREFIX)) { + continue; + } + if (funcPattern == null || funcPattern.matcher(funcName).matches()) { + funcNames.add(funcName); + } + } + return funcNames; + } + + /** + * Adds to the set of synonyms of the supplied function. + * @param funcName + * @param funcInfo + * @param synonyms + */ + public synchronized void getFunctionSynonyms( + String funcName, FunctionInfo funcInfo, Set synonyms) { + Class funcClass = funcInfo.getFunctionClass(); + for (Map.Entry entry : mFunctions.entrySet()) { + String name = entry.getKey(); + if (name.equals(funcName)) { + continue; + } + FunctionInfo function = entry.getValue(); + if (function.getFunctionClass() == funcClass) { + synonyms.add(name); + } + } + } + + /** + * Get the GenericUDAF evaluator for the name and argumentClasses. + * + * @param name the name of the UDAF + * @param argumentOIs + * @param isDistinct + * @param isAllColumns + * @return The UDAF evaluator + */ + @SuppressWarnings("deprecation") + public GenericUDAFEvaluator getGenericUDAFEvaluator(String name, + List argumentOIs, boolean isDistinct, + boolean isAllColumns) throws SemanticException { + + GenericUDAFResolver udafResolver = getGenericUDAFResolver(name); + if (udafResolver == null) { + return null; + } + + GenericUDAFEvaluator udafEvaluator; + ObjectInspector args[] = new ObjectInspector[argumentOIs.size()]; + // Can't use toArray here because Java is dumb when it comes to + // generics + arrays. + for (int ii = 0; ii < argumentOIs.size(); ++ii) { + args[ii] = argumentOIs.get(ii); + } + + GenericUDAFParameterInfo paramInfo = + new SimpleGenericUDAFParameterInfo( + args, isDistinct, isAllColumns); + if (udafResolver instanceof GenericUDAFResolver2) { + udafEvaluator = + ((GenericUDAFResolver2) udafResolver).getEvaluator(paramInfo); + } else { + udafEvaluator = udafResolver.getEvaluator(paramInfo.getParameters()); + } + return udafEvaluator; + } + + public GenericUDAFEvaluator getGenericWindowingEvaluator(String functionName, + List argumentOIs, boolean isDistinct, boolean isAllColumns) + throws SemanticException { + functionName = functionName.toLowerCase(); + WindowFunctionInfo info = getWindowFunctionInfo(functionName); + if (info == null) { + return null; + } + if (!functionName.equals(FunctionRegistry.LEAD_FUNC_NAME) && + !functionName.equals(FunctionRegistry.LAG_FUNC_NAME)) { + return getGenericUDAFEvaluator(functionName, argumentOIs, isDistinct, isAllColumns); + } + + // this must be lead/lag UDAF + ObjectInspector args[] = new ObjectInspector[argumentOIs.size()]; + GenericUDAFResolver udafResolver = info.getGenericUDAFResolver(); + GenericUDAFParameterInfo paramInfo = new SimpleGenericUDAFParameterInfo( + argumentOIs.toArray(args), isDistinct, isAllColumns); + return ((GenericUDAFResolver2) udafResolver).getEvaluator(paramInfo); + } + + public synchronized void addFunction(String functionName, FunctionInfo function) { + if (isNative ^ function.isNative()) { + throw new RuntimeException("Function " + functionName + " is not for this registry"); + } + functionName = functionName.toLowerCase(); + FunctionInfo prev = mFunctions.get(functionName); + if (prev != null && isBuiltInFunc(prev.getFunctionClass())) { + throw new RuntimeException("Function " + functionName + " is hive builtin function, " + + "which cannot be overriden."); + } + mFunctions.put(functionName, function); + if (function.isNative() && !function.isPersistent()) { + builtIns.add(function.getFunctionClass()); + } + } + + public synchronized void unregisterFunction(String functionName) throws HiveException { + functionName = functionName.toLowerCase(); + FunctionInfo fi = mFunctions.get(functionName); + if (fi != null) { + if (fi.isNative() && !fi.isPersistent()) { + throw new HiveException(ErrorMsg.DROP_NATIVE_FUNCTION.getMsg(functionName)); + } + mFunctions.remove(functionName); + builtIns.remove(fi.getFunctionClass()); + } + } + + public GenericUDAFResolver getGenericUDAFResolver(String functionName) { + FunctionInfo info = getFunctionInfo(functionName); + if (info != null) { + return info.getGenericUDAFResolver(); + } + return null; + } + + private FunctionInfo getQualifiedFunctionInfo(String functionName) { + FunctionInfo info = mFunctions.get(functionName); + // HIVE-6672: In HiveServer2 the JARs for this UDF may have been loaded by a different thread, + // and the current thread may not be able to resolve the UDF. Test for this condition + // and if necessary load the JARs in this thread. + if (isNative && info != null && info.isPersistent()) { + return registerToSessionRegistry(functionName, info); + } + return info; + } + + // called after session registry is checked + private FunctionInfo registerToSessionRegistry(String functionName, FunctionInfo function) { + FunctionInfo ret = null; + + try { + // Found UDF in metastore - now add it to the function registry + // At this point we should add any relevant jars that would be needed for the UDf. + List resources = function.getResources(); + try { + FunctionTask.addFunctionResources(resources); + } catch (Exception e) { + LOG.error("Unable to load resources for " + functionName + ":" + e, e); + return null; + } + + ClassLoader loader = Utilities.getSessionSpecifiedClassLoader(); + Class udfClass = Class.forName(function.getClassName(), true, loader); + + ret = FunctionRegistry.registerTemporaryUDF(functionName, udfClass, resources); + if (ret == null) { + LOG.error(function.getClassName() + " is not a valid UDF class and was not registered."); + } + } catch (ClassNotFoundException e) { + // Lookup of UDf class failed + LOG.error("Unable to load UDF class: " + e); + } + + return ret; + } + + private void checkFunctionClass(FunctionInfo cfi) throws ClassNotFoundException { + // This call will fail for non-generic UDFs using GenericUDFBridge + Class udfClass = cfi.getFunctionClass(); + // Even if we have a reference to the class (which will be the case for GenericUDFs), + // the classloader may not be able to resolve the class, which would mean reflection-based + // methods would fail such as for plan deserialization. Make sure this works too. + Class.forName(udfClass.getName(), true, Utilities.getSessionSpecifiedClassLoader()); + } + + public synchronized void clear() { + if (isNative) { + throw new IllegalStateException("System function registry cannot be cleared"); + } + mFunctions.clear(); + builtIns.clear(); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java index efecb05..484ff29 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/WindowFunctionInfo.java @@ -18,27 +18,27 @@ package org.apache.hadoop.hive.ql.exec; +import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver; import org.apache.hive.common.util.AnnotationUtils; +import java.util.List; + @SuppressWarnings("deprecation") -public class WindowFunctionInfo implements CommonFunctionInfo { - boolean supportsWindow = true; - boolean pivotResult = false; - boolean impliesOrder = false; - FunctionInfo fInfo; - - WindowFunctionInfo(FunctionInfo fInfo) { - assert fInfo.isGenericUDAF(); - this.fInfo = fInfo; - Class wfnCls = fInfo.getGenericUDAFResolver().getClass(); +public class WindowFunctionInfo extends FunctionInfo { + + private final boolean supportsWindow; + private final boolean pivotResult; + private final boolean impliesOrder; + + public WindowFunctionInfo(boolean isNative, String functionName, + GenericUDAFResolver resolver, List resources) { + super(isNative, functionName, resolver, resources); WindowFunctionDescription def = - AnnotationUtils.getAnnotation(wfnCls, WindowFunctionDescription.class); - if ( def != null) { - supportsWindow = def.supportsWindow(); - pivotResult = def.pivotResult(); - impliesOrder = def.impliesOrder(); - } + AnnotationUtils.getAnnotation(resolver.getClass(), WindowFunctionDescription.class); + supportsWindow = def == null ? true : def.supportsWindow(); + pivotResult = def == null ? false : def.pivotResult(); + impliesOrder = def == null ? false : def.impliesOrder(); } public boolean isSupportsWindow() { @@ -52,12 +52,4 @@ public boolean isPivotResult() { public boolean isImpliesOrder() { return impliesOrder; } - public FunctionInfo getfInfo() { - return fInfo; - } - - @Override - public Class getFunctionClass() { - return getfInfo().getFunctionClass(); - } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/SqlFunctionConverter.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/SqlFunctionConverter.java index 31f906a..34a287d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/SqlFunctionConverter.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/optiq/translator/SqlFunctionConverter.java @@ -157,7 +157,8 @@ private static FunctionInfo handleCastForParameterizedType(TypeInfo ti, Function } catch (UDFArgumentException e) { throw new RuntimeException(e); } - return new FunctionInfo(fi.isNative(),fi.getDisplayName(),(GenericUDF)udf); + return new FunctionInfo( + fi.isNative(), fi.getDisplayName(), (GenericUDF) udf, fi.getResources()); } // TODO: 1) handle Agg Func Name translation 2) is it correct to add func args diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java index e43d39f..16fd919 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java @@ -316,7 +316,7 @@ static private boolean hasUserFunctions(ExprNodeDesc expr) { if (!(expr instanceof ExprNodeGenericFuncDesc)) { return false; } - if (!FunctionRegistry.isNativeFuncExpr((ExprNodeGenericFuncDesc)expr)) { + if (!FunctionRegistry.isBuiltInFuncExpr((ExprNodeGenericFuncDesc) expr)) { return true; } for (ExprNodeDesc child : expr.getChildren()) { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java index 22e5b47..0d47c0b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.metastore.api.ResourceType; import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.FunctionUtils; import org.apache.hadoop.hive.ql.exec.TaskFactory; @@ -95,13 +96,16 @@ private void analyzeDropFunction(ASTNode ast) throws SemanticException { boolean throwException = !ifExists && !HiveConf.getBoolVar(conf, ConfVars.DROPIGNORESNONEXISTENT); - if (FunctionRegistry.getFunctionInfo(functionName) == null) { + FunctionInfo info = FunctionRegistry.getFunctionInfo(functionName); + if (info == null) { if (throwException) { throw new SemanticException(ErrorMsg.INVALID_FUNCTION.getMsg(functionName)); } else { // Fail silently return; } + } else if (info.isNative() && !info.isPersistent()) { + throw new SemanticException(ErrorMsg.DROP_NATIVE_FUNCTION.getMsg(functionName)); } boolean isTemporaryFunction = (ast.getFirstChildWithType(HiveParser.TOK_TEMPORARY) != null); diff --git ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index af633cb..c10c530 100644 --- ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; import org.apache.hadoop.hive.ql.MapRedStats; +import org.apache.hadoop.hive.ql.exec.Registry; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager; import org.apache.hadoop.hive.ql.exec.tez.TezSessionState; @@ -240,6 +241,8 @@ */ private final Set preReloadableAuxJars = new HashSet(); + private final Registry registry = new Registry(); + /** * Get the lineage state stored in this session. * @@ -721,8 +724,21 @@ public static SessionState get() { return tss.get(); } + public static Registry getRegistry() { + SessionState session = get(); + return session != null ? session.registry : null; + } + + public static Registry getRegistryForWrite() { + Registry registry = getRegistry(); + if (registry == null) { + throw new RuntimeException("Function registery for session is not initialized"); + } + return registry; + } + /** - * get hiveHitsory object which does structured logging. + * get hiveHistory object which does structured logging. * * @return The hive history object */ @@ -1228,6 +1244,7 @@ public void setCurrentDatabase(String currentDatabase) { } public void close() throws IOException { + registry.clear();; if (txnMgr != null) txnMgr.closeTxnManager(); JavaUtils.closeClassLoadersTo(conf.getClassLoader(), parentLoader); File resourceDir = diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java index aa48a6c..5c5c838 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/MatchPath.java @@ -24,10 +24,12 @@ import java.util.Map; import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; import org.apache.hadoop.hive.ql.exec.PTFPartition; import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.ASTNode; import org.apache.hadoop.hive.ql.parse.PTFTranslator; @@ -128,6 +130,7 @@ public void setInputColumnNames(HashMap inputColumnNamesMap) { this.inputColumnNamesMap = inputColumnNamesMap; } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("match_path")) public static class MatchPathResolver extends TableFunctionResolver { diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java index fcf6afd..c71da02 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/Noop.java @@ -20,8 +20,10 @@ import java.util.List; +import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.PTFPartition; import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.PTFDesc; @@ -40,6 +42,7 @@ protected void execute(PTFPartitionIterator pItr, PTFPartition oPart) { throw new UnsupportedOperationException(); } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("noop")) public static class NoopResolver extends TableFunctionResolver { @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopStreaming.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopStreaming.java index 41e0102..1de9924 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopStreaming.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopStreaming.java @@ -22,6 +22,8 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.PTFDesc; import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef; @@ -57,6 +59,7 @@ public void initializeStreaming(Configuration cfg, return rows; } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("noop_stream")) public static class NoopStreamingResolver extends NoopResolver { @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java index 0b090a9..687a044 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMap.java @@ -20,7 +20,9 @@ import java.util.ArrayList; +import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.PTFDesc; @@ -41,6 +43,7 @@ protected PTFPartition _transformRawInput(PTFPartition iPart) throws HiveExcepti return iPart; } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("noop_map")) public static class NoopWithMapResolver extends TableFunctionResolver { diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMapStreaming.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMapStreaming.java index 5d322d3..1b4f1c3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMapStreaming.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/NoopWithMapStreaming.java @@ -22,6 +22,8 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.PTFDesc; import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef; @@ -56,6 +58,7 @@ public void initializeStreaming(Configuration cfg, return rows; } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("noop_map_stream")) public static class NoopWithMapStreamingResolver extends NoopWithMapResolver { @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java index 903a9b0..3c6092f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java +++ ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java @@ -28,9 +28,11 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.PTFOperator; import org.apache.hadoop.hive.ql.exec.PTFPartition; +import org.apache.hadoop.hive.ql.exec.PartitionTableFunctionDescription; import org.apache.hadoop.hive.ql.exec.WindowFunctionInfo; import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator; import org.apache.hadoop.hive.ql.exec.PTFRollingPartition; @@ -528,6 +530,7 @@ public boolean canIterateOutput() { ArrayUtils.toPrimitive(wFnsWithWindows.toArray(new Integer[wFnsWithWindows.size()]))); } + @PartitionTableFunctionDescription(isInternal = true, description = @Description("windowin_table_func")) public static class WindowingTableFunctionResolver extends TableFunctionResolver { /* diff --git ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java index 46f8052..9d7166c 100644 --- ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java +++ ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java @@ -66,12 +66,12 @@ public void testDropMacroDoesNotExist() throws Exception { @Test public void testDropMacroExistsDoNotIgnoreErrors() throws Exception { conf.setBoolVar(ConfVars.DROPIGNORESNONEXISTENT, false); - FunctionRegistry.registerGenericUDF(false, "SOME_MACRO", GenericUDFMacro.class); + FunctionRegistry.registerTemporaryUDF("SOME_MACRO", GenericUDFMacro.class); analyze(parse("DROP TEMPORARY MACRO SOME_MACRO")); } @Test public void testDropMacro() throws Exception { - FunctionRegistry.registerGenericUDF(false, "SOME_MACRO", GenericUDFMacro.class); + FunctionRegistry.registerTemporaryUDF("SOME_MACRO", GenericUDFMacro.class); analyze(parse("DROP TEMPORARY MACRO SOME_MACRO")); } @Test(expected = SemanticException.class) diff --git ql/src/test/queries/clientnegative/drop_native_udf.q ql/src/test/queries/clientnegative/drop_native_udf.q index ae047bb..4188cf2 100644 --- ql/src/test/queries/clientnegative/drop_native_udf.q +++ ql/src/test/queries/clientnegative/drop_native_udf.q @@ -1 +1 @@ -DROP TEMPORARY FUNCTION max; +DROP FUNCTION max; diff --git ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out index c7405ed..77467f6 100644 --- ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out +++ ql/src/test/results/clientnegative/create_function_nonexistent_class.q.out @@ -2,5 +2,5 @@ PREHOOK: query: create function default.badfunc as 'my.nonexistent.class' PREHOOK: type: CREATEFUNCTION PREHOOK: Output: database:default PREHOOK: Output: default.badfunc -FAILED: Class my.nonexistent.class not found +Failed to register default.badfunc using class my.nonexistent.class FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask diff --git ql/src/test/results/clientnegative/create_function_nonudf_class.q.out ql/src/test/results/clientnegative/create_function_nonudf_class.q.out index d0dd50a..6d5427e 100644 --- ql/src/test/results/clientnegative/create_function_nonudf_class.q.out +++ ql/src/test/results/clientnegative/create_function_nonudf_class.q.out @@ -2,5 +2,5 @@ PREHOOK: query: create function default.badfunc as 'java.lang.String' PREHOOK: type: CREATEFUNCTION PREHOOK: Output: database:default PREHOOK: Output: default.badfunc -FAILED: Class java.lang.String does not implement UDF, GenericUDF, or UDAF +Failed to register default.badfunc using class java.lang.String FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask diff --git ql/src/test/results/clientnegative/drop_native_udf.q.out ql/src/test/results/clientnegative/drop_native_udf.q.out index 9f0eaa5..9e8b9ee 100644 --- ql/src/test/results/clientnegative/drop_native_udf.q.out +++ ql/src/test/results/clientnegative/drop_native_udf.q.out @@ -1,4 +1 @@ -PREHOOK: query: DROP TEMPORARY FUNCTION max -PREHOOK: type: DROPFUNCTION -PREHOOK: Output: max -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.FunctionTask +FAILED: SemanticException [Error 10297]: Cannot drop native function max diff --git service/src/test/org/apache/hadoop/hive/service/TestHiveServerSessions.java service/src/test/org/apache/hadoop/hive/service/TestHiveServerSessions.java index fd38907..c33cada 100644 --- service/src/test/org/apache/hadoop/hive/service/TestHiveServerSessions.java +++ service/src/test/org/apache/hadoop/hive/service/TestHiveServerSessions.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.service; import junit.framework.TestCase; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.transport.TSocket; @@ -38,8 +40,16 @@ private TSocket[] transports = new TSocket[clientNum]; private HiveClient[] clients = new HiveClient[clientNum]; + private String tableName; + private Path dataFilePath; + public TestHiveServerSessions(String name) { super(name); + HiveConf conf = new HiveConf(TestHiveServerSessions.class); + String dataFileDir = conf.get("test.data.files").replace('\\', '/') + .replace("c:", ""); + dataFilePath = new Path(dataFileDir, "kv1.txt"); + tableName = "testsessionfunc"; } @Override @@ -59,6 +69,7 @@ public void run() { transport.open(); transports[i] = transport; clients[i] = new HiveClient(new TBinaryProtocol(transport)); + clients[i].execute("set hive.support.concurrency = false"); } } @@ -96,4 +107,26 @@ public void testSessionVars() throws Exception { assertEquals("hiveconf:var=value" + i, clients[i].fetchOne()); } } + + public void testSessionFuncs() throws Exception { + + try { + clients[0].execute("drop table " + tableName); + } catch (Exception ex) { + } + clients[0].execute("create table " + tableName + " (key int, value string)"); + clients[0].execute("load data local inpath '" + dataFilePath.toString() + + "' into table " + tableName); + + clients[0].execute("create temporary function dummy as " + + "'org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan'"); + clients[1].execute("create temporary function dummy as " + + "'org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan'"); + + clients[0].execute("select count(*) from " + tableName + " where dummy(key, 100)"); + assertEquals(84, Integer.valueOf(clients[0].fetchOne()).intValue()); + + clients[1].execute("select count(*) from " + tableName + " where dummy(key, 100)"); + assertEquals(416, Integer.valueOf(clients[1].fetchOne()).intValue()); + } }