protected ExprNodeDesc getXpathOrFuncExprNodeDesc()

in flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParserTypeCheckProcFactory.java [966:1324]


        protected ExprNodeDesc getXpathOrFuncExprNodeDesc(
                HiveParserASTNode expr,
                boolean isFunction,
                ArrayList<ExprNodeDesc> children,
                HiveParserTypeCheckCtx ctx)
                throws SemanticException, UDFArgumentException {
            // return the child directly if the conversion is redundant.
            if (isRedundantConversionFunction(expr, isFunction, children)) {
                assert (children.size() == 1);
                assert (children.get(0) != null);
                return children.get(0);
            }
            String funcText = getFunctionText(expr, isFunction);
            ExprNodeDesc desc;
            if (funcText.equals(".")) {
                // "." : FIELD Expression

                assert (children.size() == 2);
                // Only allow constant field name for now
                assert (children.get(1) instanceof ExprNodeConstantDesc);
                ExprNodeDesc object = children.get(0);
                ExprNodeConstantDesc fieldName = (ExprNodeConstantDesc) children.get(1);
                assert (fieldName.getValue() instanceof String);

                // Calculate result TypeInfo
                String fieldNameString = (String) fieldName.getValue();
                TypeInfo objectTypeInfo = object.getTypeInfo();

                // Allow accessing a field of list element structs directly from a list
                boolean isList =
                        (object.getTypeInfo().getCategory() == ObjectInspector.Category.LIST);
                if (isList) {
                    objectTypeInfo = ((ListTypeInfo) objectTypeInfo).getListElementTypeInfo();
                }
                if (objectTypeInfo.getCategory() != ObjectInspector.Category.STRUCT) {
                    throw new SemanticException(
                            HiveParserErrorMsg.getMsg(ErrorMsg.INVALID_DOT, expr));
                }
                TypeInfo t =
                        ((StructTypeInfo) objectTypeInfo).getStructFieldTypeInfo(fieldNameString);
                if (isList) {
                    t = TypeInfoFactory.getListTypeInfo(t);
                }

                desc = new ExprNodeFieldDesc(t, children.get(0), fieldNameString, isList);
            } else if (funcText.equals("[")) {
                // "[]" : LSQUARE/INDEX Expression
                if (!ctx.getallowIndexExpr()) {
                    throw new SemanticException(
                            HiveParserErrorMsg.getMsg(ErrorMsg.INVALID_FUNCTION, expr));
                }

                assert (children.size() == 2);

                // Check whether this is a list or a map
                TypeInfo myt = children.get(0).getTypeInfo();

                if (myt.getCategory() == ObjectInspector.Category.LIST) {
                    // Only allow integer index for now
                    if (!HiveParserTypeInfoUtils.implicitConvertible(
                            children.get(1).getTypeInfo(), TypeInfoFactory.intTypeInfo)) {
                        throw new SemanticException(
                                HiveParserUtils.generateErrorMessage(
                                        expr, ErrorMsg.INVALID_ARRAYINDEX_TYPE.getMsg()));
                    }

                    // Calculate TypeInfo
                    TypeInfo t = ((ListTypeInfo) myt).getListElementTypeInfo();
                    desc =
                            new ExprNodeGenericFuncDesc(
                                    t, FunctionRegistry.getGenericUDFForIndex(), children);
                } else if (myt.getCategory() == ObjectInspector.Category.MAP) {
                    if (!HiveParserTypeInfoUtils.implicitConvertible(
                            children.get(1).getTypeInfo(),
                            ((MapTypeInfo) myt).getMapKeyTypeInfo())) {
                        throw new SemanticException(
                                HiveParserErrorMsg.getMsg(ErrorMsg.INVALID_MAPINDEX_TYPE, expr));
                    }
                    // Calculate TypeInfo
                    TypeInfo t = ((MapTypeInfo) myt).getMapValueTypeInfo();
                    desc =
                            new ExprNodeGenericFuncDesc(
                                    t, FunctionRegistry.getGenericUDFForIndex(), children);
                } else {
                    throw new SemanticException(
                            HiveParserErrorMsg.getMsg(
                                    ErrorMsg.NON_COLLECTION_TYPE, expr, myt.getTypeName()));
                }
            } else {
                // other operators or functions
                // TODO: should check SqlOperator first and ideally shouldn't be using
                // ExprNodeGenericFuncDesc at all
                FunctionInfo fi = HiveParserUtils.getFunctionInfo(funcText);

                if (fi == null) {
                    desc = convertSqlOperator(funcText, children, ctx);
                    if (desc == null) {
                        if (isFunction) {
                            throw new SemanticException(
                                    HiveParserErrorMsg.getMsg(
                                            ErrorMsg.INVALID_FUNCTION, expr.getChild(0)));
                        } else {
                            throw new SemanticException(
                                    HiveParserErrorMsg.getMsg(ErrorMsg.INVALID_FUNCTION, expr));
                        }
                    } else {
                        return desc;
                    }
                }

                // getGenericUDF() actually clones the UDF. Just call it once and reuse.
                GenericUDF genericUDF = fi.getGenericUDF();

                if (!fi.isNative()) {
                    ctx.getUnparseTranslator()
                            .addIdentifierTranslation((HiveParserASTNode) expr.getChild(0));
                }

                // Handle type casts that may contain type parameters
                if (isFunction) {
                    HiveParserASTNode funcNameNode = (HiveParserASTNode) expr.getChild(0);
                    switch (funcNameNode.getType()) {
                        case HiveASTParser.TOK_CHAR:
                            // Add type params
                            CharTypeInfo charTypeInfo =
                                    HiveASTParseUtils.getCharTypeInfo(funcNameNode);
                            if (genericUDF != null) {
                                ((SettableUDF) genericUDF).setTypeInfo(charTypeInfo);
                            }
                            break;
                        case HiveASTParser.TOK_VARCHAR:
                            VarcharTypeInfo varcharTypeInfo =
                                    HiveASTParseUtils.getVarcharTypeInfo(funcNameNode);
                            if (genericUDF != null) {
                                ((SettableUDF) genericUDF).setTypeInfo(varcharTypeInfo);
                            }
                            break;
                        case HiveASTParser.TOK_DECIMAL:
                            DecimalTypeInfo decTypeInfo =
                                    HiveASTParseUtils.getDecimalTypeTypeInfo(funcNameNode);
                            if (genericUDF != null) {
                                ((SettableUDF) genericUDF).setTypeInfo(decTypeInfo);
                            }
                            break;
                        default:
                            // Do nothing
                            break;
                    }
                }

                validateUDF(expr, isFunction, ctx, fi, genericUDF);

                // Try to infer the type of the constant only if there are two
                // nodes, one of them is column and the other is numeric const
                if (genericUDF instanceof GenericUDFBaseCompare
                        && children.size() == 2
                        && ((children.get(0) instanceof ExprNodeConstantDesc
                                        && children.get(1) instanceof ExprNodeColumnDesc)
                                || (children.get(0) instanceof ExprNodeColumnDesc
                                        && children.get(1) instanceof ExprNodeConstantDesc))) {
                    int constIdx = children.get(0) instanceof ExprNodeConstantDesc ? 0 : 1;

                    String constType = children.get(constIdx).getTypeString().toLowerCase();
                    String columnType = children.get(1 - constIdx).getTypeString().toLowerCase();
                    final PrimitiveTypeInfo colTypeInfo =
                            TypeInfoFactory.getPrimitiveTypeInfo(columnType);
                    // Try to narrow type of constant
                    Object constVal = ((ExprNodeConstantDesc) children.get(constIdx)).getValue();
                    try {
                        if (PrimitiveObjectInspectorUtils.intTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Integer(constVal.toString())));
                        } else if (PrimitiveObjectInspectorUtils.longTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Long(constVal.toString())));
                        } else if (PrimitiveObjectInspectorUtils.doubleTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Double(constVal.toString())));
                        } else if (PrimitiveObjectInspectorUtils.floatTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Float(constVal.toString())));
                        } else if (PrimitiveObjectInspectorUtils.byteTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Byte(constVal.toString())));
                        } else if (PrimitiveObjectInspectorUtils.shortTypeEntry.equals(
                                        colTypeInfo.getPrimitiveTypeEntry())
                                && (constVal instanceof Number || constVal instanceof String)) {
                            children.set(
                                    constIdx,
                                    new ExprNodeConstantDesc(new Short(constVal.toString())));
                        }
                    } catch (NumberFormatException nfe) {
                        LOG.trace("Failed to narrow type of constant", nfe);
                        if ((genericUDF instanceof GenericUDFOPEqual
                                && !NumberUtils.isNumber(constVal.toString()))) {
                            return new ExprNodeConstantDesc(false);
                        }
                    }

                    // if column type is char and constant type is string, then convert the constant
                    // to char
                    // type with padded spaces.
                    if (constType.equalsIgnoreCase(serdeConstants.STRING_TYPE_NAME)
                            && colTypeInfo instanceof CharTypeInfo) {
                        final Object originalValue =
                                ((ExprNodeConstantDesc) children.get(constIdx)).getValue();
                        final String constValue = originalValue.toString();
                        final int length = TypeInfoUtils.getCharacterLengthForType(colTypeInfo);
                        final HiveChar newValue = new HiveChar(constValue, length);
                        children.set(constIdx, new ExprNodeConstantDesc(colTypeInfo, newValue));
                    }
                }
                if (genericUDF instanceof GenericUDFOPOr) {
                    // flatten OR
                    // TODO: don't do this because older version UDF only supports 2 args
                    List<ExprNodeDesc> childrenList = new ArrayList<>(children.size());
                    for (ExprNodeDesc child : children) {
                        if (FunctionRegistry.isOpOr(child)) {
                            childrenList.addAll(child.getChildren());
                        } else {
                            childrenList.add(child);
                        }
                    }
                    desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText, children);
                } else if (genericUDF instanceof GenericUDFOPAnd) {
                    // flatten AND
                    // TODO: don't do this because older version UDF only supports 2 args
                    List<ExprNodeDesc> childrenList = new ArrayList<>(children.size());
                    for (ExprNodeDesc child : children) {
                        if (FunctionRegistry.isOpAnd(child)) {
                            childrenList.addAll(child.getChildren());
                        } else {
                            childrenList.add(child);
                        }
                    }
                    desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText, children);
                } else if (ctx.isFoldExpr() && canConvertIntoNvl(genericUDF, children)) {
                    // Rewrite CASE into NVL
                    desc =
                            ExprNodeGenericFuncDesc.newInstance(
                                    new GenericUDFNvl(),
                                    new ArrayList<>(
                                            Arrays.asList(
                                                    children.get(0),
                                                    new ExprNodeConstantDesc(false))));
                    if (Boolean.FALSE.equals(((ExprNodeConstantDesc) children.get(1)).getValue())) {
                        desc =
                                ExprNodeGenericFuncDesc.newInstance(
                                        new GenericUDFOPNot(),
                                        new ArrayList<>(Collections.singleton(desc)));
                    }
                } else if (genericUDF instanceof GenericUDFInternalInterval) {
                    // if it's Hive's internal_interval function, we change it to our own
                    // internal_interval function.
                    // see more detail in HiveGenericUDFInternalInterval
                    desc =
                            ExprNodeGenericFuncDesc.newInstance(
                                    new HiveGenericUDFInternalInterval(), funcText, children);
                } else if (genericUDF instanceof GenericUDFOPDivide && children.size() == 2) {
                    // special case for GenericUDFOPDivide
                    // if the divisor or dividend is decimal type and the other one
                    // parameter is int/long literal, the TypeInfo of the ExprNodeGenericFuncDesc
                    // may be different with inferred result type, which will cause "Mismatch of
                    // expected output data type 'DECIMAL(..)' and function's output type
                    // 'DECIMAL(..)'" in BridgingFunctionGenUtil#verifyOutputType

                    // the reason is: in here we got expected result type, which will consider
                    // int/long literal parameter as actual precision,
                    // but in the phase to infer result type phase, the
                    // int/long literal parameter will always be considered as max precision. 10 for
                    // int, and 19 for long.
                    // To fix it, in here, we also should consider the int/long literal parameter as
                    // max precision.
                    ExprNodeDesc exprNodeDesc1 = children.get(0);
                    ExprNodeDesc exprNodeDesc2 = children.get(1);
                    // if one parameter is decimal type, and the other is int or long
                    if ((isDecimalTypeInfo(exprNodeDesc1)
                                    && exprNodeDesc2 instanceof ExprNodeConstantDesc
                                    && isIntOrLongTypeInfo(exprNodeDesc2))
                            || (isDecimalTypeInfo(exprNodeDesc2)
                                    && exprNodeDesc1 instanceof ExprNodeConstantDesc
                                    && isIntOrLongTypeInfo(exprNodeDesc1))) {
                        // find which parameter we should change
                        int childToChange = isIntOrLongTypeInfo(exprNodeDesc1) ? 0 : 1;
                        // change the int/long literal parameter to decimal type with the max
                        // precision of int/long literal which is consistent to infer logic.
                        children.set(
                                childToChange,
                                new ExprNodeConstantDesc(
                                        HiveDecimalUtils.getDecimalTypeForPrimitiveCategory(
                                                (PrimitiveTypeInfo)
                                                        children.get(childToChange).getTypeInfo()),
                                        HiveDecimal.create(
                                                ((ExprNodeConstantDesc) children.get(childToChange))
                                                        .getValue()
                                                        .toString())));
                    }
                    desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText, children);
                } else {
                    desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText, children);
                }

                // If the function is deterministic and the children are constants,
                // we try to fold the expression to remove e.g. cast on constant
                if (ctx.isFoldExpr()
                        && desc instanceof ExprNodeGenericFuncDesc
                        && FunctionRegistry.isDeterministic(genericUDF)
                        && HiveParserExprNodeDescUtils.isAllConstants(children)) {
                    ExprNodeDesc constantExpr =
                            ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc) desc);
                    if (constantExpr != null
                            // if constantExpr is instanceof ExprNodeConstantDesc, we should check
                            // whether it can be folded to constant safely for some folded constant
                            // can't be converted to calcite literal currently.
                            && (!(constantExpr instanceof ExprNodeConstantDesc)
                                    || canSafeFoldToConstant(
                                            (ExprNodeConstantDesc) constantExpr))) {
                        desc = constantExpr;
                    }
                }
            }
            // UDFOPPositive is a no-op.
            // However, we still create it, and then remove it here, to make sure we
            // only allow
            // "+" for numeric types.
            if (FunctionRegistry.isOpPositive(desc)) {
                assert (desc.getChildren().size() == 1);
                desc = desc.getChildren().get(0);
            } else if (getGenericUDFClassFromExprDesc(desc) == GenericUDFOPNegative.class) {
                // UDFOPNegative should always be folded.
                assert (desc.getChildren().size() == 1);
                ExprNodeDesc input = desc.getChildren().get(0);
                if (input instanceof ExprNodeConstantDesc
                        && desc instanceof ExprNodeGenericFuncDesc) {
                    ExprNodeDesc constantExpr =
                            ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc) desc);
                    if (constantExpr != null) {
                        desc = constantExpr;
                    }
                }
            }
            assert (desc != null);
            return desc;
        }