This is an automated email from the ASF dual-hosted git repository. huajianlan pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push: new ced0a513a23 [feature](nereids) provide error location when can not find db/table/column in query (#51768) ced0a513a23 is described below commit ced0a513a23eeff01582c925bc98c1261ee7db95 Author: 924060929 <lanhuaj...@selectdb.com> AuthorDate: Thu Jun 19 20:22:09 2025 +0800 [feature](nereids) provide error location when can not find db/table/column in query (#51768) provide error location when can not find db/table/column in query ```sql select kkk from test.tbl; Unknown column 'kkk' in 'table list' in PROJECT clause(line 1, pos 7) ``` --- .../org/apache/doris/dictionary/Dictionary.java | 5 +- .../main/java/org/apache/doris/load/ExportJob.java | 2 +- .../org/apache/doris/nereids/StatementContext.java | 5 +- .../doris/nereids/analyzer/UnboundRelation.java | 77 ++++++++++++++-------- .../apache/doris/nereids/analyzer/UnboundSlot.java | 27 ++++++-- .../org/apache/doris/nereids/parser/Location.java | 54 +++++++++++++++ .../doris/nereids/parser/LogicalPlanBuilder.java | 56 +++++++++++----- .../processor/pre/PullUpSubqueryAliasToCTE.java | 2 +- .../doris/nereids/rules/analysis/BindRelation.java | 6 +- .../doris/nereids/rules/analysis/BindSink.java | 8 +-- .../nereids/rules/analysis/CollectRelation.java | 11 ++-- .../nereids/rules/analysis/ExpressionAnalyzer.java | 5 ++ .../trees/plans/commands/DeleteFromCommand.java | 2 +- .../nereids/trees/plans/commands/LoadCommand.java | 2 +- .../plans/commands/ShowConstraintsCommand.java | 3 +- .../trees/plans/commands/UpdateCommand.java | 2 +- .../plans/commands/UpdateMvByPartitionCommand.java | 2 +- .../trees/plans/commands/info/CopyIntoInfo.java | 2 +- .../commands/insert/InsertIntoTableCommand.java | 5 +- .../trees/plans/commands/insert/InsertUtils.java | 2 +- .../apache/doris/nereids/util/RelationUtil.java | 16 +++-- .../catalog/constraint/ConstraintPersistTest.java | 13 ++-- .../expression/ExpressionRewriteTestHelper.java | 2 +- .../suites/load_p0/copy_into/test_copy_into.groovy | 4 +- .../query_not_exists_table_column.groovy | 43 ++++++++++++ 25 files changed, 265 insertions(+), 91 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/dictionary/Dictionary.java b/fe/fe-core/src/main/java/org/apache/doris/dictionary/Dictionary.java index f5ceea364d2..2100a510b87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/dictionary/Dictionary.java +++ b/fe/fe-core/src/main/java/org/apache/doris/dictionary/Dictionary.java @@ -48,6 +48,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -268,7 +269,7 @@ public class Dictionary extends Table { * @return true if source table's version is newer than this dictionary's version(need update dictionary). */ public boolean hasNewerSourceVersion() { - TableIf tableIf = RelationUtil.getTable(getSourceQualifiedName(), Env.getCurrentEnv()); + TableIf tableIf = RelationUtil.getTable(getSourceQualifiedName(), Env.getCurrentEnv(), Optional.empty()); if (tableIf == null) { throw new RuntimeException(getName() + "'s source table not found"); } @@ -315,7 +316,7 @@ public class Dictionary extends Table { * if has latestInvalidVersion and the base table's data not changed, we can skip update. */ public boolean checkBaseDataValid() { - TableIf tableIf = RelationUtil.getTable(getSourceQualifiedName(), Env.getCurrentEnv()); + TableIf tableIf = RelationUtil.getTable(getSourceQualifiedName(), Env.getCurrentEnv(), Optional.empty()); if (tableIf == null) { throw new RuntimeException(getName() + "'s source table not found"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java index 18f698e1eec..e7af69bdce7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java @@ -320,7 +320,7 @@ public class ExportJob implements Writable { private LogicalPlan generateOneLogicalPlan(List<String> qualifiedTableName, List<Long> tabletIds, List<String> partitions, List<NamedExpression> selectLists) { // UnboundRelation - LogicalPlan plan = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), qualifiedTableName, + LogicalPlan plan = new UnboundRelation(null, StatementScopeIdGenerator.newRelationId(), qualifiedTableName, partitions, false, tabletIds, ImmutableList.of(), Optional.empty(), Optional.empty()); // LogicalCheckPolicy plan = new LogicalCheckPolicy<>(plan); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index f44c82e09b3..17a1abd16b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -37,6 +37,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.hint.UseMvHint; import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.rules.analysis.ColumnAliasGenerator; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -334,7 +335,7 @@ public class StatementContext implements Closeable { } /** get table by table name, try to get from information from dumpfile first */ - public TableIf getAndCacheTable(List<String> tableQualifier, TableFrom tableFrom) { + public TableIf getAndCacheTable(List<String> tableQualifier, TableFrom tableFrom, Optional<Location> location) { Map<List<String>, TableIf> tables; switch (tableFrom) { case QUERY: @@ -349,7 +350,7 @@ public class StatementContext implements Closeable { default: throw new AnalysisException("Unknown table from " + tableFrom); } - return tables.computeIfAbsent(tableQualifier, k -> RelationUtil.getTable(k, connectContext.getEnv())); + return tables.computeIfAbsent(tableQualifier, k -> RelationUtil.getTable(k, connectContext.getEnv(), location)); } public void setConnectContext(ConnectContext connectContext) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index 12d4a7c74be..9dab996e226 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -22,6 +22,7 @@ import org.apache.doris.analysis.TableSnapshot; import org.apache.doris.common.Pair; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; import org.apache.doris.nereids.trees.TableSample; @@ -42,12 +43,14 @@ import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Objects; import java.util.Optional; +import javax.annotation.Nullable; /** * Represent a relation plan node that has not been bound. */ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFuncDepsPropagation { + private final Optional<Location> location; private final List<String> nameParts; private final List<String> partNames; private final List<Long> tabletIds; @@ -62,42 +65,51 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu private final Optional<TableSnapshot> tableSnapshot; public UnboundRelation(RelationId id, List<String> nameParts) { - this(id, nameParts, Optional.empty(), Optional.empty(), ImmutableList.of(), false, ImmutableList.of(), - ImmutableList.of(), Optional.empty(), Optional.empty(), null, Optional.empty(), Optional.empty()); + this(null, id, nameParts); + } + + public UnboundRelation(Location location, RelationId id, List<String> nameParts) { + this(location, id, nameParts, Optional.empty(), Optional.empty(), + ImmutableList.of(), false, ImmutableList.of(), + ImmutableList.of(), Optional.empty(), Optional.empty(), null, + Optional.empty(), Optional.empty()); } - public UnboundRelation(RelationId id, List<String> nameParts, List<String> partNames, boolean isTempPart) { - this(id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, ImmutableList.of(), + public UnboundRelation(Location location, RelationId id, List<String> nameParts, List<String> partNames, + boolean isTempPart) { + this(location, id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), null, Optional.empty(), Optional.empty()); } - public UnboundRelation(RelationId id, List<String> nameParts, List<String> partNames, boolean isTempPart, - List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, Optional<String> indexName) { - this(id, nameParts, Optional.empty(), Optional.empty(), + public UnboundRelation(Location location, RelationId id, List<String> nameParts, List<String> partNames, + boolean isTempPart, List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, + Optional<String> indexName) { + this(location, id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, Optional.empty(), Optional.empty()); } - public UnboundRelation(RelationId id, List<String> nameParts, List<String> partNames, boolean isTempPart, - List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, Optional<String> indexName, - TableScanParams scanParams, Optional<TableSnapshot> tableSnapshot) { - this(id, nameParts, Optional.empty(), Optional.empty(), + public UnboundRelation(Location location, RelationId id, List<String> nameParts, List<String> partNames, + boolean isTempPart, List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, + Optional<String> indexName, TableScanParams scanParams, Optional<TableSnapshot> tableSnapshot) { + this(location, id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams, Optional.empty(), tableSnapshot); } - public UnboundRelation(RelationId id, List<String> nameParts, Optional<GroupExpression> groupExpression, - Optional<LogicalProperties> logicalProperties, List<String> partNames, boolean isTempPart, - List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, Optional<String> indexName) { - this(id, nameParts, groupExpression, logicalProperties, partNames, + public UnboundRelation(Location location, RelationId id, List<String> nameParts, + Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties, + List<String> partNames, boolean isTempPart, List<Long> tabletIds, List<String> hints, + Optional<TableSample> tableSample, Optional<String> indexName) { + this(location, id, nameParts, groupExpression, logicalProperties, partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, Optional.empty(), Optional.empty()); } - public UnboundRelation(RelationId id, List<String> nameParts, List<String> partNames, boolean isTempPart, - List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, Optional<String> indexName, - TableScanParams scanParams, Optional<Pair<Integer, Integer>> indexInSqlString, + public UnboundRelation(Location location, RelationId id, List<String> nameParts, List<String> partNames, + boolean isTempPart, List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, + Optional<String> indexName, TableScanParams scanParams, Optional<Pair<Integer, Integer>> indexInSqlString, Optional<TableSnapshot> tableSnapshot) { - this(id, nameParts, Optional.empty(), Optional.empty(), + this(location, id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams, indexInSqlString, tableSnapshot); } @@ -105,12 +117,14 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu /** * constructor of UnboundRelation */ - public UnboundRelation(RelationId id, List<String> nameParts, Optional<GroupExpression> groupExpression, - Optional<LogicalProperties> logicalProperties, List<String> partNames, boolean isTempPart, - List<Long> tabletIds, List<String> hints, Optional<TableSample> tableSample, Optional<String> indexName, - TableScanParams scanParams, Optional<Pair<Integer, Integer>> indexInSqlString, + public UnboundRelation(@Nullable Location location, RelationId id, List<String> nameParts, + Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties, + List<String> partNames, boolean isTempPart, List<Long> tabletIds, List<String> hints, + Optional<TableSample> tableSample, Optional<String> indexName, TableScanParams scanParams, + Optional<Pair<Integer, Integer>> indexInSqlString, Optional<TableSnapshot> tableSnapshot) { super(id, PlanType.LOGICAL_UNBOUND_RELATION, groupExpression, logicalProperties); + this.location = Optional.ofNullable(location); this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, "nameParts should not null")); this.partNames = ImmutableList.copyOf(Objects.requireNonNull(partNames, "partNames should not null")); this.tabletIds = ImmutableList.copyOf(Objects.requireNonNull(tabletIds, "tabletIds should not null")); @@ -139,7 +153,7 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu @Override public Plan withGroupExpression(Optional<GroupExpression> groupExpression) { - return new UnboundRelation(relationId, nameParts, + return new UnboundRelation(location.orElse(null), relationId, nameParts, groupExpression, Optional.of(getLogicalProperties()), partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, indexInSqlString, tableSnapshot); } @@ -147,14 +161,15 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu @Override public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties, List<Plan> children) { - return new UnboundRelation(relationId, nameParts, groupExpression, logicalProperties, partNames, - isTempPart, tabletIds, hints, tableSample, indexName, null, indexInSqlString, tableSnapshot); + return new UnboundRelation(location.orElse(null), relationId, nameParts, groupExpression, + logicalProperties, partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, + indexInSqlString, tableSnapshot); } public UnboundRelation withIndexInSql(Pair<Integer, Integer> index) { - return new UnboundRelation(relationId, nameParts, groupExpression, Optional.of(getLogicalProperties()), - partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, - Optional.of(index), tableSnapshot); + return new UnboundRelation(location.orElse(null), relationId, nameParts, groupExpression, + Optional.of(getLogicalProperties()), partNames, isTempPart, tabletIds, hints, tableSample, indexName, + null, Optional.of(index), tableSnapshot); } @Override @@ -220,4 +235,8 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu public Optional<TableSnapshot> getTableSnapshot() { return tableSnapshot; } + + public Optional<Location> getLocation() { + return location; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java index 3c5608f4f28..6b2947bdce1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundSlot.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.analyzer; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -29,30 +30,42 @@ import com.google.common.collect.Lists; import java.util.List; import java.util.Objects; import java.util.Optional; +import javax.annotation.Nullable; /** * Slot has not been bound. */ public class UnboundSlot extends Slot implements Unbound, PropagateNullable { + private Optional<Location> slotLocation; private final List<String> nameParts; public UnboundSlot(String... nameParts) { - this(ImmutableList.copyOf(nameParts), Optional.empty()); + this(null, nameParts); + } + + public UnboundSlot(@Nullable Location location, String... nameParts) { + this(location, ImmutableList.copyOf(nameParts), Optional.empty()); } public UnboundSlot(List<String> nameParts) { - this(ImmutableList.copyOf(nameParts), Optional.empty()); + this(null, nameParts, Optional.empty()); + } + + public UnboundSlot(@Nullable Location location, List<String> nameParts) { + this(location, ImmutableList.copyOf(nameParts), Optional.empty()); } - public UnboundSlot(List<String> nameParts, Optional<Pair<Integer, Integer>> indexInSqlString) { + public UnboundSlot( + @Nullable Location location, List<String> nameParts, Optional<Pair<Integer, Integer>> indexInSqlString) { super(indexInSqlString); this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, "nameParts can not be null")); + this.slotLocation = Optional.ofNullable(location); } @Override public Slot withIndexInSql(Pair<Integer, Integer> index) { - return new UnboundSlot(nameParts, Optional.ofNullable(index)); + return new UnboundSlot(slotLocation.orElse(null), nameParts, Optional.ofNullable(index)); } public List<String> getNameParts() { @@ -91,7 +104,7 @@ public class UnboundSlot extends Slot implements Unbound, PropagateNullable { } public static UnboundSlot quoted(String name) { - return new UnboundSlot(Lists.newArrayList(name), Optional.empty()); + return new UnboundSlot(null, Lists.newArrayList(name), Optional.empty()); } @Override @@ -116,6 +129,10 @@ public class UnboundSlot extends Slot implements Unbound, PropagateNullable { return nameParts.hashCode(); } + public Optional<Location> getLocation() { + return slotLocation; + } + @Override public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) { return visitor.visitUnboundSlot(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Location.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Location.java new file mode 100644 index 00000000000..628128f1ff5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Location.java @@ -0,0 +1,54 @@ +// 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.doris.nereids.parser; + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; + +/** Location: the AST location in the SQL */ +public class Location { + public final int line; + public final int pos; + + public Location(int line, int pos) { + this.line = line; + this.pos = pos; + } + + @Override + public String toString() { + return "line " + line + ", pos " + pos; + } + + /** fromToken */ + public static Location fromToken(Token token) { + if (token == null) { + return null; + } + return new Location(token.getLine(), token.getCharPositionInLine()); + } + + /** fromContext */ + public static Location fromAst(ParserRuleContext parserRuleContext) { + if (parserRuleContext == null) { + return null; + } + Token start = parserRuleContext.getStart(); + return new Location(start.getLine(), start.getCharPositionInLine()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 730bb3e13d9..8b93c346097 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1635,9 +1635,10 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { @Override public LogicalPlan visitAddConstraint(AddConstraintContext ctx) { List<String> parts = visitMultipartIdentifier(ctx.table); - UnboundRelation curTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), parts); - ImmutableList<Slot> slots = visitIdentifierList(ctx.constraint().slots).stream() - .map(UnboundSlot::new) + UnboundRelation curTable = new UnboundRelation( + Location.fromAst(ctx.table), StatementScopeIdGenerator.newRelationId(), parts); + ImmutableList<Slot> slots = ctx.constraint().slots.identifierSeq().ident.stream() + .map(ident -> new UnboundSlot(Location.fromAst(ident), ident.getText())) .collect(ImmutableList.toImmutableList()); Constraint constraint; if (ctx.constraint().UNIQUE() != null) { @@ -1645,11 +1646,15 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { } else if (ctx.constraint().PRIMARY() != null) { constraint = Constraint.newPrimaryKeyConstraint(curTable, slots); } else if (ctx.constraint().FOREIGN() != null) { - ImmutableList<Slot> referencedSlots = visitIdentifierList(ctx.constraint().referencedSlots).stream() - .map(UnboundSlot::new) + ImmutableList<Slot> referencedSlots = ctx.constraint().referencedSlots.identifierSeq().ident.stream() + .map(ident -> new UnboundSlot(Location.fromAst(ident), ident.getText())) .collect(ImmutableList.toImmutableList()); List<String> nameParts = visitMultipartIdentifier(ctx.constraint().referenceTable); - LogicalPlan referenceTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), nameParts); + LogicalPlan referenceTable = new UnboundRelation( + Location.fromAst(ctx.constraint().referenceTable), + StatementScopeIdGenerator.newRelationId(), + nameParts + ); constraint = Constraint.newForeignKeyConstraint(curTable, slots, referenceTable, referencedSlots); } else { throw new AnalysisException("Unsupported constraint " + ctx.getText()); @@ -1660,14 +1665,20 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { @Override public LogicalPlan visitDropConstraint(DropConstraintContext ctx) { List<String> parts = visitMultipartIdentifier(ctx.table); - UnboundRelation curTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), parts); + UnboundRelation curTable = new UnboundRelation( + Location.fromAst(ctx.table), StatementScopeIdGenerator.newRelationId(), parts); return new DropConstraintCommand(ctx.constraintName.getText().toLowerCase(), curTable); } @Override public LogicalPlan visitUpdate(UpdateContext ctx) { - LogicalPlan query = LogicalPlanBuilderAssistant.withCheckPolicy(new UnboundRelation( - StatementScopeIdGenerator.newRelationId(), visitMultipartIdentifier(ctx.tableName))); + LogicalPlan query = LogicalPlanBuilderAssistant.withCheckPolicy( + new UnboundRelation( + Location.fromAst(ctx.tableName), + StatementScopeIdGenerator.newRelationId(), + visitMultipartIdentifier(ctx.tableName) + ) + ); query = withTableAlias(query, ctx.tableAlias()); if (ctx.fromClause() != null) { query = withRelations(query, ctx.fromClause().relations().relation()); @@ -1694,8 +1705,11 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { throw new ParseException("Now don't support auto detect partitions in deleting", ctx); } LogicalPlan query = withTableAlias(LogicalPlanBuilderAssistant.withCheckPolicy( - new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableName, - partitionSpec.second, partitionSpec.first)), ctx.tableAlias()); + new UnboundRelation( + Location.fromAst(ctx.tableName), + StatementScopeIdGenerator.newRelationId(), tableName, + partitionSpec.second, partitionSpec.first)), ctx.tableAlias() + ); String tableAlias = null; if (ctx.tableAlias().strictIdentifier() != null) { tableAlias = ctx.tableAlias().getText(); @@ -1948,7 +1962,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { .collect(ImmutableList.toImmutableList()); Function unboundFunction = new UnboundFunction(functionName, arguments); return new LogicalGenerate<>(ImmutableList.of(unboundFunction), - ImmutableList.of(new UnboundSlot(generateName, columnName)), ImmutableList.of(expandColumnNames), plan); + ImmutableList.of(new UnboundSlot(Location.fromAst(ctx.columnNames.get(0)), generateName, columnName)), + ImmutableList.of(expandColumnNames), plan); } /** @@ -2350,7 +2365,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { } TableSample tableSample = ctx.sample() == null ? null : (TableSample) visit(ctx.sample()); - UnboundRelation relation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), + UnboundRelation relation = new UnboundRelation( + Location.fromAst(ctx.multipartIdentifier()), StatementScopeIdGenerator.newRelationId(), nameParts, partitionNames, isTempPart, tabletIdLists, relationHints, Optional.ofNullable(tableSample), indexName, scanParams, Optional.ofNullable(tableSnapshot)); @@ -3088,7 +3104,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { UnboundSlot unboundAttribute = (UnboundSlot) e; List<String> nameParts = Lists.newArrayList(unboundAttribute.getNameParts()); nameParts.add(ctx.fieldName.getText()); - UnboundSlot slot = new UnboundSlot(nameParts, Optional.empty()); + UnboundSlot slot = new UnboundSlot(Location.fromAst(ctx.fieldName), nameParts, Optional.empty()); return slot; } else { // todo: base is an expression, may be not a table name. @@ -3114,7 +3130,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { @Override public Expression visitColumnReference(ColumnReferenceContext ctx) { // todo: handle quoted and unquoted - return UnboundSlot.quoted(ctx.getText()); + return new UnboundSlot(Location.fromAst(ctx), Lists.newArrayList(ctx.getText()), Optional.empty()); } /** @@ -3308,7 +3324,9 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { @Override public EqualTo visitUpdateAssignment(UpdateAssignmentContext ctx) { - return new EqualTo(new UnboundSlot(visitMultipartIdentifier(ctx.multipartIdentifier()), Optional.empty()), + return new EqualTo(new UnboundSlot( + Location.fromAst(ctx.multipartIdentifier()), + visitMultipartIdentifier(ctx.multipartIdentifier()), Optional.empty()), getExpression(ctx.expression())); } @@ -3481,7 +3499,11 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { .map(partition -> { IdentifierContext identifier = partition.identifier(); if (identifier != null) { - return UnboundSlot.quoted(identifier.getText()); + return new UnboundSlot( + Location.fromAst(identifier), + Lists.newArrayList(identifier.getText()), + Optional.empty() + ); } else { return visitFunctionCallExpression(partition.functionCallExpression()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java index 31a205d5ed5..cae945cbf1d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java @@ -63,7 +63,7 @@ public class PullUpSubqueryAliasToCTE extends PlanPreprocessor { aliasQueries.add((LogicalSubQueryAlias<Plan>) alias); List<String> tableName = new ArrayList<>(); tableName.add(alias.getAlias()); - return new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableName); + return new UnboundRelation(null, StatementScopeIdGenerator.newRelationId(), tableName); } return alias; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 9b024de1f4f..fe4b47bd8f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -164,7 +164,8 @@ public class BindRelation extends OneAnalysisRuleFactory { } List<String> tableQualifier = RelationUtil.getQualifierName( cascadesContext.getConnectContext(), unboundRelation.getNameParts()); - TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY, + unboundRelation.getLocation()); LogicalPlan scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); if (cascadesContext.isLeadingJoin()) { @@ -178,7 +179,8 @@ public class BindRelation extends OneAnalysisRuleFactory { private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboundRelation) { List<String> tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), unboundRelation.getNameParts()); - TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY, + unboundRelation.getLocation()); return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 7617977f00b..b2a23c8ad59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -706,7 +706,7 @@ public class BindSink implements AnalysisRuleFactory { List<String> tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), sink.getNameParts()); Pair<DatabaseIf<?>, TableIf> pair = RelationUtil.getDbAndTable(tableQualifier, - cascadesContext.getConnectContext().getEnv()); + cascadesContext.getConnectContext().getEnv(), Optional.empty()); if (!(pair.second instanceof OlapTable)) { throw new AnalysisException("the target table of insert into is not an OLAP table"); } @@ -718,7 +718,7 @@ public class BindSink implements AnalysisRuleFactory { List<String> tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), sink.getNameParts()); Pair<DatabaseIf<?>, TableIf> pair = RelationUtil.getDbAndTable(tableQualifier, - cascadesContext.getConnectContext().getEnv()); + cascadesContext.getConnectContext().getEnv(), Optional.empty()); if (pair.second instanceof HMSExternalTable) { HMSExternalTable table = (HMSExternalTable) pair.second; if (table.getDlaType() == HMSExternalTable.DLAType.HIVE) { @@ -733,7 +733,7 @@ public class BindSink implements AnalysisRuleFactory { List<String> tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), sink.getNameParts()); Pair<DatabaseIf<?>, TableIf> pair = RelationUtil.getDbAndTable(tableQualifier, - cascadesContext.getConnectContext().getEnv()); + cascadesContext.getConnectContext().getEnv(), Optional.empty()); if (pair.second instanceof IcebergExternalTable) { return Pair.of(((IcebergExternalDatabase) pair.first), (IcebergExternalTable) pair.second); } @@ -745,7 +745,7 @@ public class BindSink implements AnalysisRuleFactory { List<String> tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), sink.getNameParts()); Pair<DatabaseIf<?>, TableIf> pair = RelationUtil.getDbAndTable(tableQualifier, - cascadesContext.getConnectContext().getEnv()); + cascadesContext.getConnectContext().getEnv(), Optional.empty()); if (pair.second instanceof JdbcExternalTable) { return Pair.of(((JdbcExternalDatabase) pair.first), (JdbcExternalTable) pair.second); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java index c5a3eedf62a..0e668e94da3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.pattern.MatchingContext; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -140,7 +141,7 @@ public class CollectRelation implements AnalysisRuleFactory { case 3: // catalog.db.table // Use catalog and database name from name parts. - collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.INSERT_TARGET); + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.INSERT_TARGET, Optional.empty()); return null; default: throw new IllegalStateException("Insert target name is invalid."); @@ -159,7 +160,7 @@ public class CollectRelation implements AnalysisRuleFactory { case 3: // catalog.db.table // Use catalog and database name from name parts. - collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.QUERY); + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.QUERY, ctx.root.getLocation()); return null; default: throw new IllegalStateException("Table name [" + ctx.root.getTableName() + "] is invalid."); @@ -167,7 +168,7 @@ public class CollectRelation implements AnalysisRuleFactory { } private void collectFromUnboundRelation(CascadesContext cascadesContext, - List<String> nameParts, TableFrom tableFrom) { + List<String> nameParts, TableFrom tableFrom, Optional<Location> location) { if (nameParts.size() == 1) { String tableName = nameParts.get(0); // check if it is a CTE's name @@ -186,7 +187,7 @@ public class CollectRelation implements AnalysisRuleFactory { table = ((UnboundDictionarySink) cascadesContext.getRewritePlan()).getDictionary(); } else { table = cascadesContext.getConnectContext().getStatementContext() - .getAndCacheTable(tableQualifier, tableFrom); + .getAndCacheTable(tableQualifier, tableFrom, location); } LOG.info("collect table {} from {}", nameParts, tableFrom); if (tableFrom == TableFrom.QUERY) { @@ -232,7 +233,7 @@ public class CollectRelation implements AnalysisRuleFactory { } try { cascadesContext.getStatementContext().getAndCacheTable(baseTableInfo.toList(), - TableFrom.MTMV); + TableFrom.MTMV, Optional.empty()); } catch (AnalysisException exception) { LOG.warn("mtmv related base table get err, related table is {}", baseTableInfo.toList(), exception); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java index 50b00d0d2b3..482f1fc1979 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java @@ -36,6 +36,7 @@ import org.apache.doris.nereids.analyzer.UnboundStar; import org.apache.doris.nereids.analyzer.UnboundVariable; import org.apache.doris.nereids.analyzer.UnboundVariable.VariableType; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.rules.expression.AbstractExpressionRewriteRule; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; @@ -323,6 +324,10 @@ public class ExpressionAnalyzer extends SubExprAnalyzer<ExpressionRewriteContext if (currentPlan != null) { message += "' in " + currentPlan.getType().toString().substring("LOGICAL_".length()) + " clause"; } + Optional<Location> columnLocation = unboundSlot.getLocation(); + if (columnLocation.isPresent()) { + message += "(" + columnLocation.get() + ")"; + } throw new AnalysisException(message); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java index 146e03189a3..ecff74bc3bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java @@ -450,7 +450,7 @@ public class DeleteFromCommand extends Command implements ForwardWithSync, Expla private OlapTable getTargetTable(ConnectContext ctx) { List<String> qualifiedTableName = RelationUtil.getQualifierName(ctx, nameParts); - TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv()); + TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv(), Optional.empty()); if (!(table instanceof OlapTable)) { throw new AnalysisException("table must be olapTable in delete command"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java index 78fe220fa23..dd0577b6905 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java @@ -444,7 +444,7 @@ public class LoadCommand extends Command implements NeedAuditEncryption, Forward private static OlapTable getOlapTable(ConnectContext ctx, BulkLoadDataDesc dataDesc) throws AnalysisException { OlapTable targetTable; - TableIf table = RelationUtil.getTable(dataDesc.getNameParts(), ctx.getEnv()); + TableIf table = RelationUtil.getTable(dataDesc.getNameParts(), ctx.getEnv(), Optional.empty()); if (!(table instanceof OlapTable)) { throw new AnalysisException("table must be olapTable in load command"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java index 0e5c332058d..5c66e3a67dc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java @@ -30,6 +30,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -51,7 +52,7 @@ public class ShowConstraintsCommand extends Command implements NoForward { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf tableIf = RelationUtil.getDbAndTable( - RelationUtil.getQualifierName(ctx, nameParts), ctx.getEnv()).value(); + RelationUtil.getQualifierName(ctx, nameParts), ctx.getEnv(), Optional.empty()).value(); tableIf.readLock(); List<List<String>> res; try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java index ada4ca9035c..05391ba3ca7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java @@ -228,7 +228,7 @@ public class UpdateCommand extends Command implements ForwardWithSync, Explainab + ctx.getSessionVariable().printDebugModeVariables()); } List<String> tableQualifier = RelationUtil.getQualifierName(ctx, nameParts); - TableIf table = RelationUtil.getTable(tableQualifier, ctx.getEnv()); + TableIf table = RelationUtil.getTable(tableQualifier, ctx.getEnv(), Optional.empty()); if (!(table instanceof OlapTable)) { throw new AnalysisException("target table in update command should be an olapTable"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index fac66f97abb..2e4577fda40 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -242,7 +242,7 @@ public class UpdateMvByPartitionCommand extends InsertOverwriteTableCommand { } List<String> tableQualifier = RelationUtil.getQualifierName(ConnectContext.get(), unboundRelation.getNameParts()); - TableIf table = RelationUtil.getTable(tableQualifier, Env.getCurrentEnv()); + TableIf table = RelationUtil.getTable(tableQualifier, Env.getCurrentEnv(), Optional.empty()); if (predicates.getPredicates().containsKey(table)) { return new LogicalFilter<>(ImmutableSet.of(ExpressionUtils.or(predicates.getPredicates().get(table))), unboundRelation); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CopyIntoInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CopyIntoInfo.java index 8ed9095fe91..090ae7175ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CopyIntoInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CopyIntoInfo.java @@ -204,7 +204,7 @@ public class CopyIntoInfo { List<String> nameParts = Lists.newArrayList(); nameParts.add(db); nameParts.add(tableName.getTbl()); - Plan unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), nameParts); + Plan unboundRelation = new UnboundRelation(null, StatementScopeIdGenerator.newRelationId(), nameParts); CascadesContext cascadesContext = CascadesContext.initContext(ConnectContext.get().getStatementContext(), unboundRelation, PhysicalProperties.ANY); Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 1f843b2dc57..8e5d65b9a27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -173,8 +173,9 @@ public class InsertIntoTableCommand extends Command implements NeedAuditEncrypti } // may be overridden - protected TableIf getTargetTableIf(ConnectContext ctx, List<String> qualifiedTargetTableName) { - return RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); + protected TableIf getTargetTableIf( + ConnectContext ctx, List<String> qualifiedTargetTableName) { + return RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv(), Optional.empty()); } public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor) throws Exception { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java index 92e6f4826b0..009a66661bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java @@ -547,7 +547,7 @@ public class InsertUtils { */ public static TableIf getTargetTable(Plan plan, ConnectContext ctx) { List<String> tableQualifier = getTargetTableQualified(plan, ctx); - return RelationUtil.getTable(tableQualifier, ctx.getEnv()); + return RelationUtil.getTable(tableQualifier, ctx.getEnv(), Optional.empty()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java index 75a2fa8ffdf..983b411e135 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java @@ -27,6 +27,7 @@ import org.apache.doris.datasource.systable.SysTable; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.parser.Location; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Slot; @@ -109,14 +110,15 @@ public class RelationUtil { /** * get table */ - public static TableIf getTable(List<String> qualifierName, Env env) { - return getDbAndTable(qualifierName, env).second; + public static TableIf getTable(List<String> qualifierName, Env env, Optional<Location> location) { + return getDbAndTable(qualifierName, env, location).second; } /** * get database and table */ - public static Pair<DatabaseIf<?>, TableIf> getDbAndTable(List<String> qualifierName, Env env) { + public static Pair<DatabaseIf<?>, TableIf> getDbAndTable( + List<String> qualifierName, Env env, Optional<Location> location) { String catalogName = qualifierName.get(0); String dbName = qualifierName.get(1); String tableName = qualifierName.get(2); @@ -126,12 +128,16 @@ public class RelationUtil { } try { DatabaseIf<TableIf> db = catalog.getDbOrException(dbName, s -> new AnalysisException( - "Database [" + dbName + "] does not exist.")); + "Database [" + dbName + "] does not exist." + + (location.map(loc -> "(" + loc + ")").orElse(""))) + ); Pair<String, String> tableNameWithSysTableName = SysTable.getTableNameWithSysTableName(tableName); TableIf tbl = db.getTableOrException(tableNameWithSysTableName.first, s -> new AnalysisException( - "Table [" + tableName + "] does not exist in database [" + dbName + "].")); + "Table [" + tableName + "] does not exist in database [" + dbName + "]." + + (location.map(loc -> "(" + loc + ")").orElse(""))) + ); Optional<TableValuedFunction> sysTable = tbl.getSysTableFunction(catalogName, dbName, tableName); if (!Strings.isNullOrEmpty(tableNameWithSysTableName.second) && !sysTable.isPresent()) { throw new AnalysisException("Unknown sys table '" + tableName + "'"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java index a38b5f49fc0..c169c7bc341 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/constraint/ConstraintPersistTest.java @@ -54,6 +54,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatchSupported { @@ -91,7 +92,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); Map<String, Constraint> constraintMap = tableIf.getConstraintsMap(); tableIf.getConstraintsMapUnsafe().clear(); Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); @@ -126,7 +127,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); Map<String, Constraint> constraintMap = tableIf.getConstraintsMap(); ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); DataOutput output = new DataOutputStream(outputStream); @@ -176,7 +177,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); Assertions.assertEquals(3, tableIf.getConstraintsMap().size()); dropConstraint("alter table t1 drop constraint uk"); dropConstraint("alter table t1 drop constraint pk"); @@ -200,7 +201,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table t1 add constraint fk foreign key (k1) references t2(k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); DataOutput output = new DataOutputStream(outputStream); tableIf.write(output); @@ -271,7 +272,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); Map<String, Constraint> constraintMap = tableIf.getConstraintsMap(); tableIf.getConstraintsMapUnsafe().clear(); Assertions.assertTrue(tableIf.getConstraintsMap().isEmpty()); @@ -333,7 +334,7 @@ class ConstraintPersistTest extends TestWithFeService implements PlanPatternMatc addConstraint("alter table es.es_db1.es_tbl1 add constraint uk unique (k1)"); TableIf tableIf = RelationUtil.getTable( RelationUtil.getQualifierName(connectContext, Lists.newArrayList("test", "t1")), - connectContext.getEnv()); + connectContext.getEnv(), Optional.empty()); Map<String, Constraint> constraintMap = tableIf.getConstraintsMap(); ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); DataOutput output = new DataOutputStream(outputStream); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java index 3763c6145d8..6634f986423 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java @@ -58,7 +58,7 @@ public abstract class ExpressionRewriteTestHelper extends ExpressionRewrite { public ExpressionRewriteTestHelper() { CascadesContext cascadesContext = MemoTestUtils.createCascadesContext( - new UnboundRelation(new RelationId(1), ImmutableList.of("tbl"))); + new UnboundRelation(null, new RelationId(1), ImmutableList.of("tbl"))); context = new ExpressionRewriteContext(cascadesContext); } diff --git a/regression-test/suites/load_p0/copy_into/test_copy_into.groovy b/regression-test/suites/load_p0/copy_into/test_copy_into.groovy index ff1cb7bdb83..64e448731ca 100644 --- a/regression-test/suites/load_p0/copy_into/test_copy_into.groovy +++ b/regression-test/suites/load_p0/copy_into/test_copy_into.groovy @@ -103,10 +103,10 @@ suite("test_copy_into", "p0") { def errorMsgs = [ "", - "errCode = 2, detailMessage = In where clause '(p_type = not_exist)', unknown column 'not_exist' in 'table list", + "errCode = 2, detailMessage = In where clause '(p_type = not_exist)', unknown column 'not_exist' in 'table list(line 3, pos 65)", "", "", - "errCode = 2, detailMessage = In where clause '(p_type = not_exist)', unknown column 'not_exist' in 'table list", + "errCode = 2, detailMessage = In where clause '(p_type = not_exist)', unknown column 'not_exist' in 'table list(line 3, pos 65)", "", "", "", diff --git a/regression-test/suites/nereids_syntax_p0/query_not_exists_table_column.groovy b/regression-test/suites/nereids_syntax_p0/query_not_exists_table_column.groovy new file mode 100644 index 00000000000..733bbe7169e --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/query_not_exists_table_column.groovy @@ -0,0 +1,43 @@ +// 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. + + +suite("query_not_exists_table_column") { + test { + sql("select * from fasdfasdf.kfafds") + exception("Database [fasdfasdf] does not exist.(line 1, pos 14)") + } + + def currentDb = (sql "select database()")[0][0] + test { + sql("select * from ${currentDb}.asdfasfdsaf") + exception("Table [asdfasfdsaf] does not exist in database [${currentDb}].(line 1, pos 14)") + } + + multi_sql """ + drop table if exists query_not_exists_table_column; + create table if not exists query_not_exists_table_column( + id int + ) + distributed by hash(id) + properties('replication_num'='1')""" + + test { + sql("select kkk from ${currentDb}.query_not_exists_table_column") + exception("Unknown column 'kkk' in 'table list' in PROJECT clause(line 1, pos 7)") + } +} \ No newline at end of file --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org