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

Reply via email to