[ 
https://issues.apache.org/jira/browse/FLINK-33788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17794964#comment-17794964
 ] 

lishaoguang commented on FLINK-33788:
-------------------------------------

源码修改:
org.apache.flink.table.planner.delegation.hive.copy.HiveParserSemanticAnalyzer

{code:java}
private String processTable(HiveParserQB qb, HiveParserASTNode tabref) throws 
SemanticException {
        int[] indexes = HiveParserBaseSemanticAnalyzer.findTabRefIdxs(tabref);
        int aliasIndex = indexes[0];
        int propsIndex = indexes[1];
        int tsampleIndex = indexes[2];
        int ssampleIndex = indexes[3];
        HiveParserASTNode tableTree = 
(HiveParserASTNode)((HiveParserASTNode)tabref.getChild(0));
        String qualifiedTableName = 
HiveParserBaseSemanticAnalyzer.getUnescapedName(tableTree, 
this.catalogManager.getCurrentCatalog(), 
this.catalogManager.getCurrentDatabase()){color:red}.toLowerCase(){color};
        String originTableName = 
HiveParserBaseSemanticAnalyzer.getUnescapedOriginTableName(tableTree);
        String alias = 
HiveParserBaseSemanticAnalyzer.findSimpleTableName(tabref, aliasIndex);
        if (propsIndex >= 0) {
            Tree propsAST = tabref.getChild(propsIndex);
            Map<String, String> props = 
HiveParserDDLSemanticAnalyzer.getProps((HiveParserASTNode)propsAST.getChild(0));
            if ("TRUE".equals(props.get("insideView"))) {
                qb.getAliasInsideView().add(alias.toLowerCase());
            }

            qb.setTabProps(alias, props);
        }

        if (qb.exists(alias)) {
            throw new 
SemanticException(HiveParserErrorMsg.getMsg(ErrorMsg.AMBIGUOUS_TABLE_ALIAS, 
tabref.getChild(aliasIndex)));
        } else {
            HiveParserASTNode sampleClause;
            if (tsampleIndex >= 0) {
                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));
                    }
                }

                if (sampleCols.size() > 2) {
                    throw new 
SemanticException(HiveParserUtils.generateErrorMessage((HiveParserASTNode)tabref.getChild(0),
 ErrorMsg.SAMPLE_RESTRICTION.getMsg()));
                }

                qb.getParseInfo().setTabSample(alias);
                if (this.unparseTranslator.isEnabled()) {
                    Iterator var25 = sampleCols.iterator();

                    while(var25.hasNext()) {
                        HiveParserASTNode sampleCol = 
(HiveParserASTNode)var25.next();
                        
this.unparseTranslator.addIdentifierTranslation((HiveParserASTNode)sampleCol.getChild(0));
                    }
                }
            } else if (ssampleIndex >= 0) {
                sampleClause = (HiveParserASTNode)tabref.getChild(ssampleIndex);
                Tree type = sampleClause.getChild(0);
                Tree numerator = sampleClause.getChild(1);
                String value = 
HiveParserBaseSemanticAnalyzer.unescapeIdentifier(numerator.getText());
                SplitSample sample;
                if (type.getType() == 861) {
                    double percent = Double.parseDouble(value);
                    if (percent < 0.0D || percent > 100.0D) {
                        throw new 
SemanticException(HiveParserUtils.generateErrorMessage((HiveParserASTNode)numerator,
 "Sampling percentage should be between 0 and 100"));
                    }

                    int seedNum = 
this.conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
                    sample = new SplitSample(percent, seedNum);
                } else if (type.getType() == 902) {
                    sample = new SplitSample(Integer.parseInt(value));
                } else {
                    assert type.getType() == 810;

                    long length = (long)Integer.parseInt(value.substring(0, 
value.length() - 1));
                    char last = value.charAt(value.length() - 1);
                    if (last != 'k' && last != 'K') {
                        if (last != 'm' && last != 'M') {
                            if (last == 'g' || last == 'G') {
                                length <<= 30;
                            }
                        } else {
                            length <<= 20;
                        }
                    } else {
                        length <<= 10;
                    }

                    int seedNum = 
this.conf.getIntVar(ConfVars.HIVESAMPLERANDOMNUM);
                    sample = new SplitSample(length, seedNum);
                }

                String aliasId = 
HiveParserBaseSemanticAnalyzer.getAliasId(alias, qb);
                this.nameToSplitSample.put(aliasId, sample);
            }

            qb.setTabAlias(alias, originTableName, qualifiedTableName);
            if (qb.isInsideView()) {
                qb.getAliasInsideView().add(alias.toLowerCase());
            }

            qb.addAlias(alias);
            qb.getParseInfo().setSrcForAlias(alias, tableTree);
            if (!this.aliasToCTEs.containsKey(qualifiedTableName)) {
                this.unparseTranslator.addTableNameTranslation(tableTree, 
this.catalogManager.getCurrentCatalog(), 
this.catalogManager.getCurrentDatabase());
                if (aliasIndex != 0) {
                    
this.unparseTranslator.addIdentifierTranslation((HiveParserASTNode)tabref.getChild(aliasIndex));
                }
            }

            return alias;
        }
    }
{code}


> Table not found for catalogname is capital
> ------------------------------------------
>
>                 Key: FLINK-33788
>                 URL: https://issues.apache.org/jira/browse/FLINK-33788
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 1.16.1
>            Reporter: lishaoguang
>            Priority: Major
>         Attachments: image-2023-12-09-20-26-41-150.png
>
>
> {code:java}
>         String runMode=args[0];
>         String sql = "select * from student";
>         String sql = "insert into table student select * from student;";
>         String sql = args[1];
>         String catalogName = "myHive";
>         String defaultDatabase = args[2];
>         RuntimeExecutionMode executionMode = "batch".equals(runMode) ? 
> RuntimeExecutionMode.BATCH : RuntimeExecutionMode.STREAMING;
>         StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>         env.setRuntimeMode(executionMode);
>         StreamTableEnvironment tenv = StreamTableEnvironment.create(env);
>         
>         tenv.getConfig().setSqlDialect(SqlDialect.HIVE);
>         Configuration conf = new Configuration();
>         conf.addResource("/ect/hadoop/conf");
>         conf.addResource("/ect/hive/conf");
>         HiveConf hiveConf = new HiveConf(conf, Configuration.class);
>         String hiveVersion = "3.1.3";
>         HiveCatalog hiveCatalog = new HiveCatalog(catalogName, 
> defaultDatabase, hiveConf, hiveVersion, true);
>         tenv.registerCatalog(catalogName, hiveCatalog);
>         tenv.useCatalog(catalogName);
>         tenv.useDatabase(defaultDatabase);
>         tenv.executeSql(sql).print(); {code}
> Even if the table exists in hive, the query will still report an error 
> like"Caused by: org.apache.hadoop.hive.ql.parse.SemanticException: Line 1:14 
> Table not found 'student'"



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to