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]