private String processTable()

in flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/copy/HiveParserSemanticAnalyzer.java [463:587]


    private String processTable(HiveParserQB qb, HiveParserASTNode tabref)
            throws SemanticException {
        // For each table reference get the table name
        // and the alias (if alias is not present, the table name
        // is used as an alias)
        int[] indexes = findTabRefIdxs(tabref);
        int aliasIndex = indexes[0];
        int propsIndex = indexes[1];
        int tsampleIndex = indexes[2];
        int ssampleIndex = indexes[3];

        HiveParserASTNode tableTree = (HiveParserASTNode) (tabref.getChild(0));

        String qualifiedTableName =
                getUnescapedName(
                                tableTree,
                                catalogRegistry.getCurrentCatalog(),
                                catalogRegistry.getCurrentDatabase())
                        .toLowerCase();
        String originTableName = getUnescapedOriginTableName(tableTree);

        String alias = findSimpleTableName(tabref, aliasIndex);

        if (propsIndex >= 0) {
            Tree propsAST = tabref.getChild(propsIndex);
            Map<String, String> props =
                    HiveParserDDLSemanticAnalyzer.getProps(
                            (HiveParserASTNode) propsAST.getChild(0));
            // We get the information from Calcite.
            if ("TRUE".equals(props.get("insideView"))) {
                qb.getAliasInsideView().add(alias.toLowerCase());
            }
            qb.setTabProps(alias, props);
        }

        // If the alias is already there then we have a conflict
        if (qb.exists(alias)) {
            throw new SemanticException(
                    HiveParserErrorMsg.getMsg(
                            ErrorMsg.AMBIGUOUS_TABLE_ALIAS, tabref.getChild(aliasIndex)));
        }
        if (tsampleIndex >= 0) {
            HiveParserASTNode sampleClause = (HiveParserASTNode) tabref.getChild(tsampleIndex);
            ArrayList<HiveParserASTNode> sampleCols = new ArrayList<>();
            if (sampleClause.getChildCount() > 2) {
                for (int i = 2; i < sampleClause.getChildCount(); i++) {
                    sampleCols.add((HiveParserASTNode) sampleClause.getChild(i));
                }
            }
            // TODO: For now only support sampling on up to two columns
            // Need to change it to list of columns
            if (sampleCols.size() > 2) {
                throw new SemanticException(
                        HiveParserUtils.generateErrorMessage(
                                (HiveParserASTNode) tabref.getChild(0),
                                ErrorMsg.SAMPLE_RESTRICTION.getMsg()));
            }
            qb.getParseInfo().setTabSample(alias);
            if (unparseTranslator.isEnabled()) {
                for (HiveParserASTNode sampleCol : sampleCols) {
                    unparseTranslator.addIdentifierTranslation(
                            (HiveParserASTNode) sampleCol.getChild(0));
                }
            }
        } else if (ssampleIndex >= 0) {
            HiveParserASTNode sampleClause = (HiveParserASTNode) tabref.getChild(ssampleIndex);

            Tree type = sampleClause.getChild(0);
            Tree numerator = sampleClause.getChild(1);
            String value = unescapeIdentifier(numerator.getText());

            SplitSample sample;
            if (type.getType() == HiveASTParser.TOK_PERCENT) {
                double percent = Double.parseDouble(value);
                if (percent < 0 || percent > 100) {
                    throw new SemanticException(
                            HiveParserUtils.generateErrorMessage(
                                    (HiveParserASTNode) numerator,
                                    "Sampling percentage should be between 0 and 100"));
                }
                int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
                sample = new SplitSample(percent, seedNum);
            } else if (type.getType() == HiveASTParser.TOK_ROWCOUNT) {
                sample = new SplitSample(Integer.parseInt(value));
            } else {
                assert type.getType() == HiveASTParser.TOK_LENGTH;
                long length = Integer.parseInt(value.substring(0, value.length() - 1));
                char last = value.charAt(value.length() - 1);
                if (last == 'k' || last == 'K') {
                    length <<= 10;
                } else if (last == 'm' || last == 'M') {
                    length <<= 20;
                } else if (last == 'g' || last == 'G') {
                    length <<= 30;
                }
                int seedNum = conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
                sample = new SplitSample(length, seedNum);
            }
            String aliasId = getAliasId(alias, qb);
            nameToSplitSample.put(aliasId, sample);
        }
        // Insert this map into the stats
        qb.setTabAlias(alias, originTableName, qualifiedTableName);
        if (qb.isInsideView()) {
            qb.getAliasInsideView().add(alias.toLowerCase());
        }
        qb.addAlias(alias);

        qb.getParseInfo().setSrcForAlias(alias, tableTree);

        // if alias to CTE contains the table name, we do not do the translation because
        // cte is actually a subquery.
        if (!this.aliasToCTEs.containsKey(qualifiedTableName)) {
            unparseTranslator.addTableNameTranslation(
                    tableTree,
                    catalogRegistry.getCurrentCatalog(),
                    catalogRegistry.getCurrentDatabase());
            if (aliasIndex != 0) {
                unparseTranslator.addIdentifierTranslation(
                        (HiveParserASTNode) tabref.getChild(aliasIndex));
            }
        }

        return alias;
    }