This is an automated email from the ASF dual-hosted git repository.

morrysnow 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 f24c94bf505 [refactor](nereids) Support Origin to provide error 
location (#52125)
f24c94bf505 is described below

commit f24c94bf5052352a82860da4eb5e83c18830f862
Author: 924060929 <[email protected]>
AuthorDate: Tue Jun 24 17:56:36 2025 +0800

    [refactor](nereids) Support Origin to provide error location (#52125)
    
    ### What problem does this PR solve?
    
    follow up #51768
    1. use `Origin` to provide error location, instead of `Location`, so we
    can pass Origin to TreeNode without change the constructor
    2. try best to access `Origin` by `MoreFieldsThread.origin` to speed up
    10x, compare to `ThreadLocal<Origin>`
---
 .../org/apache/doris/common/ThreadPoolManager.java | 10 +++-
 .../main/java/org/apache/doris/load/ExportJob.java |  2 +-
 .../java/org/apache/doris/mysql/MysqlServer.java   |  1 +
 .../org/apache/doris/nereids/StatementContext.java |  8 ++-
 .../doris/nereids/analyzer/UnboundRelation.java    | 42 +++++---------
 .../apache/doris/nereids/analyzer/UnboundSlot.java | 27 ++-------
 .../org/apache/doris/nereids/parser/Location.java  | 54 -----------------
 .../doris/nereids/parser/LogicalPlanBuilder.java   | 23 +++-----
 .../org/apache/doris/nereids/parser/Origin.java    |  9 +++
 .../apache/doris/nereids/parser/ParserUtils.java   | 52 ++++++++++++++++-
 .../processor/pre/PullUpSubqueryAliasToCTE.java    |  2 +-
 .../doris/nereids/rules/analysis/BindRelation.java |  4 +-
 .../nereids/rules/analysis/CollectRelation.java    |  7 +--
 .../nereids/rules/analysis/ExpressionAnalyzer.java |  8 +--
 .../doris/nereids/trees/AbstractTreeNode.java      | 10 ++++
 .../org/apache/doris/nereids/trees/TreeNode.java   |  5 ++
 .../trees/plans/commands/info/CopyIntoInfo.java    |  2 +-
 .../doris/nereids/util/MoreFieldsThread.java       | 67 ++++++++++++++++++++++
 .../apache/doris/nereids/util/RelationUtil.java    | 15 +++--
 .../expression/ExpressionRewriteTestHelper.java    |  2 +-
 20 files changed, 205 insertions(+), 145 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java 
b/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java
index 0822a322dec..33f0c627085 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java
@@ -23,6 +23,7 @@ import org.apache.doris.metric.Metric;
 import org.apache.doris.metric.Metric.MetricUnit;
 import org.apache.doris.metric.MetricLabel;
 import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.nereids.util.MoreFieldsThread;
 
 import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -240,10 +241,13 @@ public class ThreadPoolManager {
      * Create a thread factory that names threads with a prefix and also sets 
the threads to daemon.
      */
     private static ThreadFactory namedThreadFactory(String poolName) {
-        return new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat(poolName + "-%d").build();
+        return new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat(poolName + "-%d")
+                .setThreadFactory(MoreFieldsThread::new)
+                .build();
     }
 
-
     public static ThreadPoolExecutor newDaemonThreadPoolWithPreAuth(
             int corePoolSize,
             int maximumPoolSize,
@@ -267,7 +271,7 @@ public class ThreadPoolManager {
         return new ThreadFactoryBuilder()
             .setDaemon(true)
             .setNameFormat(poolName + "-%d")
-            .setThreadFactory(runnable -> new Thread(() -> {
+            .setThreadFactory(runnable -> new MoreFieldsThread(() -> {
                 try {
                     preAuth.execute(runnable);
                 } catch (Exception e) {
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 e7af69bdce7..18f698e1eec 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(null, 
StatementScopeIdGenerator.newRelationId(), qualifiedTableName,
+        LogicalPlan plan = new 
UnboundRelation(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/mysql/MysqlServer.java 
b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlServer.java
index e7a888cdd24..be3ba9296db 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlServer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlServer.java
@@ -51,6 +51,7 @@ public class MysqlServer {
     private AcceptingChannel<StreamConnection> server;
 
     // default task service.
+
     private ExecutorService taskService = 
ThreadPoolManager.newDaemonCacheThreadPoolThrowException(
             Config.max_mysql_service_task_threads_num, "mysql-nio-pool", true);
 
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 104204b89d7..58e7b000345 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
@@ -34,11 +34,11 @@ import org.apache.doris.datasource.mvcc.MvccSnapshot;
 import org.apache.doris.datasource.mvcc.MvccTable;
 import org.apache.doris.datasource.mvcc.MvccTableInfo;
 import org.apache.doris.mtmv.BaseTableInfo;
+import org.apache.doris.nereids.analyzer.UnboundRelation;
 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;
@@ -341,7 +341,8 @@ 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, Optional<Location> location) {
+    public TableIf getAndCacheTable(List<String> tableQualifier, TableFrom 
tableFrom,
+            Optional<UnboundRelation> unboundRelation) {
         Map<List<String>, TableIf> tables;
         switch (tableFrom) {
             case QUERY:
@@ -356,7 +357,8 @@ public class StatementContext implements Closeable {
             default:
                 throw new AnalysisException("Unknown table from " + tableFrom);
         }
-        return tables.computeIfAbsent(tableQualifier, k -> 
RelationUtil.getTable(k, connectContext.getEnv(), location));
+        return tables.computeIfAbsent(
+                tableQualifier, k -> RelationUtil.getTable(k, 
connectContext.getEnv(), unboundRelation));
     }
 
     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 9dab996e226..b271b7a791f 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,7 +22,6 @@ 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;
@@ -43,14 +42,12 @@ 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;
@@ -65,51 +62,47 @@ public class UnboundRelation extends LogicalRelation 
implements Unbound, BlockFu
     private final Optional<TableSnapshot> tableSnapshot;
 
     public UnboundRelation(RelationId id, List<String> nameParts) {
-        this(null, id, nameParts);
-    }
-
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts) {
-        this(location, id, nameParts, Optional.empty(), Optional.empty(),
+        this(id, nameParts, Optional.empty(), Optional.empty(),
                 ImmutableList.of(), false, ImmutableList.of(),
                 ImmutableList.of(), Optional.empty(), Optional.empty(), null,
                 Optional.empty(), Optional.empty());
     }
 
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts, List<String> partNames,
+    public UnboundRelation(RelationId id, List<String> nameParts, List<String> 
partNames,
             boolean isTempPart) {
-        this(location, id, nameParts, Optional.empty(), Optional.empty(), 
partNames, isTempPart, ImmutableList.of(),
+        this(id, nameParts, Optional.empty(), Optional.empty(), partNames, 
isTempPart, ImmutableList.of(),
                 ImmutableList.of(), Optional.empty(), Optional.empty(), null, 
Optional.empty(), Optional.empty());
     }
 
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts, List<String> partNames,
+    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(location, id, nameParts, Optional.empty(), Optional.empty(),
+        this(id, nameParts, Optional.empty(), Optional.empty(),
                 partNames, isTempPart, tabletIds, hints, tableSample, 
indexName, null, Optional.empty(),
                 Optional.empty());
     }
 
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts, List<String> partNames,
+    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(location, id, nameParts, Optional.empty(), Optional.empty(),
+        this(id, nameParts, Optional.empty(), Optional.empty(),
                 partNames, isTempPart, tabletIds, hints, tableSample, 
indexName, scanParams, Optional.empty(),
                 tableSnapshot);
     }
 
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts,
+    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(location, id, nameParts, groupExpression, logicalProperties, 
partNames,
+        this(id, nameParts, groupExpression, logicalProperties, partNames,
                 isTempPart, tabletIds, hints, tableSample, indexName, null, 
Optional.empty(), Optional.empty());
     }
 
-    public UnboundRelation(Location location, RelationId id, List<String> 
nameParts, List<String> partNames,
+    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,
             Optional<TableSnapshot> tableSnapshot) {
-        this(location, id, nameParts, Optional.empty(), Optional.empty(),
+        this(id, nameParts, Optional.empty(), Optional.empty(),
                 partNames, isTempPart, tabletIds, hints, tableSample, 
indexName, scanParams, indexInSqlString,
                 tableSnapshot);
     }
@@ -117,14 +110,13 @@ public class UnboundRelation extends LogicalRelation 
implements Unbound, BlockFu
     /**
      * constructor of UnboundRelation
      */
-    public UnboundRelation(@Nullable Location location, RelationId id, 
List<String> nameParts,
+    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,
             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"));
@@ -153,7 +145,7 @@ public class UnboundRelation extends LogicalRelation 
implements Unbound, BlockFu
 
     @Override
     public Plan withGroupExpression(Optional<GroupExpression> groupExpression) 
{
-        return new UnboundRelation(location.orElse(null), relationId, 
nameParts,
+        return new UnboundRelation(relationId, nameParts,
                 groupExpression, Optional.of(getLogicalProperties()),
                 partNames, isTempPart, tabletIds, hints, tableSample, 
indexName, null, indexInSqlString, tableSnapshot);
     }
@@ -161,13 +153,13 @@ public class UnboundRelation extends LogicalRelation 
implements Unbound, BlockFu
     @Override
     public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> 
groupExpression,
             Optional<LogicalProperties> logicalProperties, List<Plan> 
children) {
-        return new UnboundRelation(location.orElse(null), relationId, 
nameParts, groupExpression,
+        return new UnboundRelation(relationId, nameParts, groupExpression,
                 logicalProperties, partNames, isTempPart, tabletIds, hints, 
tableSample, indexName, null,
                 indexInSqlString, tableSnapshot);
     }
 
     public UnboundRelation withIndexInSql(Pair<Integer, Integer> index) {
-        return new UnboundRelation(location.orElse(null), relationId, 
nameParts, groupExpression,
+        return new UnboundRelation(relationId, nameParts, groupExpression,
                 Optional.of(getLogicalProperties()), partNames, isTempPart, 
tabletIds, hints, tableSample, indexName,
                 null, Optional.of(index), tableSnapshot);
     }
@@ -235,8 +227,4 @@ 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 6b2947bdce1..9f60aa2e68e 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,7 +18,6 @@
 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;
@@ -30,42 +29,30 @@ 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(null, nameParts);
-    }
-
-    public UnboundSlot(@Nullable Location location, String... nameParts) {
-        this(location, ImmutableList.copyOf(nameParts), Optional.empty());
+        this(ImmutableList.copyOf(nameParts), Optional.empty());
     }
 
     public UnboundSlot(List<String> nameParts) {
-        this(null, nameParts, Optional.empty());
-    }
-
-    public UnboundSlot(@Nullable Location location, List<String> nameParts) {
-        this(location, ImmutableList.copyOf(nameParts), Optional.empty());
+        this(Utils.fastToImmutableList(nameParts), Optional.empty());
     }
 
-    public UnboundSlot(
-            @Nullable Location location, List<String> nameParts, 
Optional<Pair<Integer, Integer>> indexInSqlString) {
+    public UnboundSlot(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(slotLocation.orElse(null), nameParts, 
Optional.ofNullable(index));
+        return new UnboundSlot(nameParts, Optional.ofNullable(index));
     }
 
     public List<String> getNameParts() {
@@ -104,7 +91,7 @@ public class UnboundSlot extends Slot implements Unbound, 
PropagateNullable {
     }
 
     public static UnboundSlot quoted(String name) {
-        return new UnboundSlot(null, Lists.newArrayList(name), 
Optional.empty());
+        return new UnboundSlot(Lists.newArrayList(name), Optional.empty());
     }
 
     @Override
@@ -129,10 +116,6 @@ 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
deleted file mode 100644
index 628128f1ff5..00000000000
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Location.java
+++ /dev/null
@@ -1,54 +0,0 @@
-// 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 d708e3e3b69..2fcc94dbfbc 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
@@ -1653,10 +1653,9 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     @Override
     public LogicalPlan visitAddConstraint(AddConstraintContext ctx) {
         List<String> parts = visitMultipartIdentifier(ctx.table);
-        UnboundRelation curTable = new UnboundRelation(
-                Location.fromAst(ctx.table), 
StatementScopeIdGenerator.newRelationId(), parts);
+        UnboundRelation curTable = new 
UnboundRelation(StatementScopeIdGenerator.newRelationId(), parts);
         ImmutableList<Slot> slots = 
ctx.constraint().slots.identifierSeq().ident.stream()
-                .map(ident -> new UnboundSlot(Location.fromAst(ident), 
ident.getText()))
+                .map(ident -> new UnboundSlot(ident.getText()))
                 .collect(ImmutableList.toImmutableList());
         Constraint constraint;
         if (ctx.constraint().UNIQUE() != null) {
@@ -1665,11 +1664,10 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
             constraint = Constraint.newPrimaryKeyConstraint(curTable, slots);
         } else if (ctx.constraint().FOREIGN() != null) {
             ImmutableList<Slot> referencedSlots = 
ctx.constraint().referencedSlots.identifierSeq().ident.stream()
-                    .map(ident -> new UnboundSlot(Location.fromAst(ident), 
ident.getText()))
+                    .map(ident -> new UnboundSlot(ident.getText()))
                     .collect(ImmutableList.toImmutableList());
             List<String> nameParts = 
visitMultipartIdentifier(ctx.constraint().referenceTable);
             LogicalPlan referenceTable = new UnboundRelation(
-                    Location.fromAst(ctx.constraint().referenceTable),
                     StatementScopeIdGenerator.newRelationId(),
                     nameParts
             );
@@ -1683,8 +1681,7 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     @Override
     public LogicalPlan visitDropConstraint(DropConstraintContext ctx) {
         List<String> parts = visitMultipartIdentifier(ctx.table);
-        UnboundRelation curTable = new UnboundRelation(
-                Location.fromAst(ctx.table), 
StatementScopeIdGenerator.newRelationId(), parts);
+        UnboundRelation curTable = new 
UnboundRelation(StatementScopeIdGenerator.newRelationId(), parts);
         return new 
DropConstraintCommand(ctx.constraintName.getText().toLowerCase(), curTable);
     }
 
@@ -1692,7 +1689,6 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     public LogicalPlan visitUpdate(UpdateContext ctx) {
         LogicalPlan query = LogicalPlanBuilderAssistant.withCheckPolicy(
                 new UnboundRelation(
-                        Location.fromAst(ctx.tableName),
                         StatementScopeIdGenerator.newRelationId(),
                         visitMultipartIdentifier(ctx.tableName)
                 )
@@ -1724,7 +1720,6 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
         }
         LogicalPlan query = 
withTableAlias(LogicalPlanBuilderAssistant.withCheckPolicy(
                 new UnboundRelation(
-                        Location.fromAst(ctx.tableName),
                         StatementScopeIdGenerator.newRelationId(), tableName,
                         partitionSpec.second, partitionSpec.first)), 
ctx.tableAlias()
         );
@@ -1980,7 +1975,7 @@ 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(Location.fromAst(ctx.columnNames.get(0)), generateName, 
columnName)),
+                ImmutableList.of(new UnboundSlot(generateName, columnName)),
                 ImmutableList.of(expandColumnNames), plan);
     }
 
@@ -2384,7 +2379,7 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
 
         TableSample tableSample = ctx.sample() == null ? null : (TableSample) 
visit(ctx.sample());
         UnboundRelation relation = new UnboundRelation(
-                Location.fromAst(ctx.multipartIdentifier()), 
StatementScopeIdGenerator.newRelationId(),
+                StatementScopeIdGenerator.newRelationId(),
                 nameParts, partitionNames, isTempPart, tabletIdLists, 
relationHints,
                 Optional.ofNullable(tableSample), indexName, scanParams, 
Optional.ofNullable(tableSnapshot));
 
@@ -3122,7 +3117,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(Location.fromAst(ctx.fieldName), nameParts, Optional.empty());
+                UnboundSlot slot = new UnboundSlot(nameParts, 
Optional.empty());
                 return slot;
             } else {
                 // todo: base is an expression, may be not a table name.
@@ -3148,7 +3143,7 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     @Override
     public Expression visitColumnReference(ColumnReferenceContext ctx) {
         // todo: handle quoted and unquoted
-        return new UnboundSlot(Location.fromAst(ctx), 
Lists.newArrayList(ctx.getText()), Optional.empty());
+        return new UnboundSlot(Lists.newArrayList(ctx.getText()), 
Optional.empty());
     }
 
     /**
@@ -3343,7 +3338,6 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
     @Override
     public EqualTo visitUpdateAssignment(UpdateAssignmentContext ctx) {
         return new EqualTo(new UnboundSlot(
-                Location.fromAst(ctx.multipartIdentifier()),
                 visitMultipartIdentifier(ctx.multipartIdentifier()), 
Optional.empty()),
                 getExpression(ctx.expression()));
     }
@@ -3518,7 +3512,6 @@ public class LogicalPlanBuilder extends 
DorisParserBaseVisitor<Object> {
                     IdentifierContext identifier = partition.identifier();
                     if (identifier != null) {
                         return new UnboundSlot(
-                                Location.fromAst(identifier),
                                 Lists.newArrayList(identifier.getText()),
                                 Optional.empty()
                         );
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Origin.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Origin.java
index f5c24be6f8c..0b9bd528d9f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Origin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/Origin.java
@@ -34,4 +34,13 @@ public class Origin {
         this.line = line;
         this.startPosition = startPosition;
     }
+
+    @Override
+    public String toString() {
+        if (line.isPresent()) {
+            return "line " + line.get() + ", pos " + startPosition.get();
+        } else {
+            return "unknown position";
+        }
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParserUtils.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParserUtils.java
index c829fe01c35..b36a5eb4729 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParserUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParserUtils.java
@@ -17,19 +17,69 @@
 
 package org.apache.doris.nereids.parser;
 
+import org.apache.doris.nereids.util.MoreFieldsThread;
+
 import org.antlr.v4.runtime.CharStream;
 import org.antlr.v4.runtime.ParserRuleContext;
 import org.antlr.v4.runtime.Token;
 import org.antlr.v4.runtime.misc.Interval;
 
+import java.util.Optional;
 import java.util.function.Supplier;
 
 /**
  * Utils for parser.
  */
 public class ParserUtils {
+    private static final ThreadLocal<Origin> slowThreadLocal = new 
ThreadLocal<>();
+
+    /** getOrigin */
+    public static Optional<Origin> getOrigin() {
+        Thread thread = Thread.currentThread();
+        Origin origin;
+        if (thread instanceof MoreFieldsThread) {
+            // fast path
+            origin = ((MoreFieldsThread) thread).getOrigin();
+        } else {
+            // slow path
+            origin = slowThreadLocal.get();
+        }
+        return Optional.ofNullable(origin);
+    }
+
+    /** withOrigin */
     public static <T> T withOrigin(ParserRuleContext ctx, Supplier<T> f) {
-        return f.get();
+        Token startToken = ctx.getStart();
+        Origin origin = new Origin(
+                Optional.of(startToken.getLine()),
+                Optional.of(startToken.getCharPositionInLine())
+        );
+
+        Thread thread = Thread.currentThread();
+        if (thread instanceof MoreFieldsThread) {
+            // fast path
+            MoreFieldsThread moreFieldsThread = (MoreFieldsThread) thread;
+            Origin outerOrigin = moreFieldsThread.getOrigin();
+            try {
+                moreFieldsThread.setOrigin(origin);
+                return f.get();
+            } finally {
+                moreFieldsThread.setOrigin(outerOrigin);
+            }
+        } else {
+            // slow path
+            Origin outerOrigin = slowThreadLocal.get();
+            try {
+                slowThreadLocal.set(origin);
+                return f.get();
+            } finally {
+                if (outerOrigin != null) {
+                    slowThreadLocal.set(outerOrigin);
+                } else {
+                    slowThreadLocal.remove();
+                }
+            }
+        }
     }
 
     public static String command(ParserRuleContext ctx) {
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 cae945cbf1d..31a205d5ed5 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(null, 
StatementScopeIdGenerator.newRelationId(), tableName);
+            return new 
UnboundRelation(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 d098a42aad1..a1f79d8c7bd 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
@@ -167,7 +167,7 @@ public class BindRelation extends OneAnalysisRuleFactory {
         List<String> tableQualifier = RelationUtil.getQualifierName(
                 cascadesContext.getConnectContext(), 
unboundRelation.getNameParts());
         TableIf table = 
cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, 
TableFrom.QUERY,
-                unboundRelation.getLocation());
+                Optional.of(unboundRelation));
 
         LogicalPlan scan = getLogicalPlan(table, unboundRelation, 
tableQualifier, cascadesContext);
         if (cascadesContext.isLeadingJoin()) {
@@ -182,7 +182,7 @@ public class BindRelation extends OneAnalysisRuleFactory {
         List<String> tableQualifier = 
RelationUtil.getQualifierName(cascadesContext.getConnectContext(),
                 unboundRelation.getNameParts());
         TableIf table = 
cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, 
TableFrom.QUERY,
-                unboundRelation.getLocation());
+                Optional.of(unboundRelation));
         return getLogicalPlan(table, unboundRelation, tableQualifier, 
cascadesContext);
     }
 
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 11f8844781d..87f3ad8dc50 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,7 +32,6 @@ 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;
@@ -160,7 +159,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, ctx.root.getLocation());
+                collectFromUnboundRelation(ctx.cascadesContext, nameParts, 
TableFrom.QUERY, Optional.of(ctx.root));
                 return null;
             default:
                 throw new IllegalStateException("Table name [" + 
ctx.root.getTableName() + "] is invalid.");
@@ -168,7 +167,7 @@ public class CollectRelation implements AnalysisRuleFactory 
{
     }
 
     private void collectFromUnboundRelation(CascadesContext cascadesContext,
-            List<String> nameParts, TableFrom tableFrom, Optional<Location> 
location) {
+            List<String> nameParts, TableFrom tableFrom, 
Optional<UnboundRelation> unboundRelation) {
         if (nameParts.size() == 1) {
             String tableName = nameParts.get(0);
             // check if it is a CTE's name
@@ -187,7 +186,7 @@ public class CollectRelation implements AnalysisRuleFactory 
{
             table = ((UnboundDictionarySink) 
cascadesContext.getRewritePlan()).getDictionary();
         } else {
             table = cascadesContext.getConnectContext().getStatementContext()
-                .getAndCacheTable(tableQualifier, tableFrom, location);
+                .getAndCacheTable(tableQualifier, tableFrom, unboundRelation);
         }
         LOG.info("collect table {} from {}", nameParts, tableFrom);
         if (tableFrom == TableFrom.QUERY) {
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 482f1fc1979..fe1b4ea307b 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,7 +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.parser.Origin;
 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;
@@ -324,9 +324,9 @@ 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() + ")";
+        Optional<Origin> origin = unboundSlot.getOrigin();
+        if (origin.isPresent()) {
+            message += "(" + origin.get() + ")";
         }
         throw new AnalysisException(message);
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
index 92bbcdb9b38..f608bb89cb8 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/AbstractTreeNode.java
@@ -17,6 +17,8 @@
 
 package org.apache.doris.nereids.trees;
 
+import org.apache.doris.nereids.parser.Origin;
+import org.apache.doris.nereids.parser.ParserUtils;
 import org.apache.doris.nereids.util.MutableState;
 import org.apache.doris.nereids.util.MutableState.EmptyMutableState;
 import org.apache.doris.nereids.util.Utils;
@@ -32,6 +34,9 @@ import java.util.Optional;
  */
 public abstract class AbstractTreeNode<NODE_TYPE extends TreeNode<NODE_TYPE>>
         implements TreeNode<NODE_TYPE> {
+
+    protected final Optional<Origin> origin = ParserUtils.getOrigin();
+
     protected final List<NODE_TYPE> children;
 
     // this field is special, because other fields in tree node is immutable, 
but in some scenes, mutable
@@ -53,6 +58,11 @@ public abstract class AbstractTreeNode<NODE_TYPE extends 
TreeNode<NODE_TYPE>>
         this.children = Utils.fastToImmutableList(children);
     }
 
+    @Override
+    public Optional<Origin> getOrigin() {
+        return origin;
+    }
+
     @Override
     public NODE_TYPE child(int index) {
         return children.get(index);
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
index 2c5decb9f9a..5235ad94316 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/TreeNode.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.nereids.trees;
 
+import org.apache.doris.nereids.parser.Origin;
 import org.apache.doris.nereids.util.Utils;
 
 import com.google.common.collect.ImmutableList;
@@ -48,6 +49,10 @@ public interface TreeNode<NODE_TYPE extends 
TreeNode<NODE_TYPE>> {
 
     NODE_TYPE child(int index);
 
+    default Optional<Origin> getOrigin() {
+        return Optional.empty();
+    }
+
     int arity();
 
     <T> Optional<T> getMutableState(String key);
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 090ae7175ab..8ed9095fe91 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(null, 
StatementScopeIdGenerator.newRelationId(), nameParts);
+        Plan unboundRelation = new 
UnboundRelation(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/util/MoreFieldsThread.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MoreFieldsThread.java
new file mode 100644
index 00000000000..48fa9f9a9de
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/MoreFieldsThread.java
@@ -0,0 +1,67 @@
+// 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.util;
+
+import org.apache.doris.nereids.parser.Origin;
+
+/**
+ * This class is used to extend some thread local fields for Thread,
+ * so we can access the thread fields faster than ThreadLocal
+ */
+public class MoreFieldsThread extends Thread {
+    private Origin origin;
+
+    public MoreFieldsThread() {
+    }
+
+    public MoreFieldsThread(Runnable target) {
+        super(target);
+    }
+
+    public MoreFieldsThread(ThreadGroup group, Runnable target) {
+        super(group, target);
+    }
+
+    public MoreFieldsThread(String name) {
+        super(name);
+    }
+
+    public MoreFieldsThread(ThreadGroup group, String name) {
+        super(group, name);
+    }
+
+    public MoreFieldsThread(Runnable target, String name) {
+        super(target, name);
+    }
+
+    public MoreFieldsThread(ThreadGroup group, Runnable target, String name) {
+        super(group, target, name);
+    }
+
+    public MoreFieldsThread(ThreadGroup group, Runnable target, String name, 
long stackSize) {
+        super(group, target, name, stackSize);
+    }
+
+    public final void setOrigin(Origin origin) {
+        this.origin = origin;
+    }
+
+    public final Origin getOrigin() {
+        return this.origin;
+    }
+}
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 983b411e135..e796149b748 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
@@ -26,10 +26,12 @@ import org.apache.doris.datasource.CatalogIf;
 import org.apache.doris.datasource.systable.SysTable;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.StatementContext;
+import org.apache.doris.nereids.analyzer.UnboundRelation;
 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.parser.Origin;
 import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.AbstractTreeNode;
 import org.apache.doris.nereids.trees.expressions.Slot;
 import 
org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction;
 import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
@@ -110,15 +112,15 @@ public class RelationUtil {
     /**
      * get table
      */
-    public static TableIf getTable(List<String> qualifierName, Env env, 
Optional<Location> location) {
-        return getDbAndTable(qualifierName, env, location).second;
+    public static TableIf getTable(List<String> qualifierName, Env env, 
Optional<UnboundRelation> unboundRelation) {
+        return getDbAndTable(qualifierName, env, unboundRelation).second;
     }
 
     /**
      * get database and table
      */
     public static Pair<DatabaseIf<?>, TableIf> getDbAndTable(
-            List<String> qualifierName, Env env, Optional<Location> location) {
+            List<String> qualifierName, Env env, Optional<UnboundRelation> 
unboundRelation) {
         String catalogName = qualifierName.get(0);
         String dbName = qualifierName.get(1);
         String tableName = qualifierName.get(2);
@@ -126,17 +128,18 @@ public class RelationUtil {
         if (catalog == null) {
             throw new AnalysisException(java.lang.String.format("Catalog %s 
does not exist.", catalogName));
         }
+        Optional<Origin> origin = 
unboundRelation.flatMap(AbstractTreeNode::getOrigin);
         try {
             DatabaseIf<TableIf> db = catalog.getDbOrException(dbName, s -> new 
AnalysisException(
                     "Database [" + dbName + "] does not exist."
-                            + (location.map(loc -> "(" + loc + 
")").orElse("")))
+                            + (origin.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 + "]."
-                                    + (location.map(loc -> "(" + loc + 
")").orElse("")))
+                                    + (origin.map(loc -> "(" + loc + 
")").orElse("")))
             );
             Optional<TableValuedFunction> sysTable = 
tbl.getSysTableFunction(catalogName, dbName, tableName);
             if (!Strings.isNullOrEmpty(tableNameWithSysTableName.second) && 
!sysTable.isPresent()) {
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 6634f986423..3763c6145d8 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(null, new RelationId(1), 
ImmutableList.of("tbl")));
+                new UnboundRelation(new RelationId(1), 
ImmutableList.of("tbl")));
         context = new ExpressionRewriteContext(cascadesContext);
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to