[enhancement](Nereids) New optimizer support check column privileges (#31700)

* [enhancement](Nereids) New optimizer support check column privileges (#31494)

(cherry picked from commit accfcfcae5a3343a9e2b7241453e56738255de1b)

* column privilege

* column privilege
This commit is contained in:
924060929
2024-03-03 13:48:31 +08:00
committed by GitHub
parent de9b5f7b69
commit 47d330c74d
20 changed files with 845 additions and 69 deletions

View File

@ -123,7 +123,6 @@ public class CascadesContext implements ScheduleContext {
private boolean isLeadingDisableJoinReorder = false;
private final Map<String, Hint> hintMap = Maps.newLinkedHashMap();
private final boolean shouldCheckRelationAuthentication;
private final ThreadLocal<Boolean> showPlanProcess = new ThreadLocal<>();
// This list is used to listen the change event of the plan which
@ -133,12 +132,12 @@ public class CascadesContext implements ScheduleContext {
/**
* Constructor of OptimizerContext.
*
* @param memo {@link Memo} reference
* @param statementContext {@link StatementContext} reference
* @param memo {@link Memo} reference
*/
private CascadesContext(Optional<CascadesContext> parent, Optional<CTEId> currentTree,
StatementContext statementContext, Plan plan, Memo memo,
CTEContext cteContext, PhysicalProperties requireProperties, boolean shouldCheckRelationAuthentication) {
CTEContext cteContext, PhysicalProperties requireProperties) {
this.parent = Objects.requireNonNull(parent, "parent should not null");
this.currentTree = Objects.requireNonNull(currentTree, "currentTree should not null");
this.statementContext = Objects.requireNonNull(statementContext, "statementContext should not null");
@ -152,7 +151,6 @@ public class CascadesContext implements ScheduleContext {
this.subqueryExprIsAnalyzed = new HashMap<>();
this.runtimeFilterContext = new RuntimeFilterContext(getConnectContext().getSessionVariable());
this.materializationContexts = new ArrayList<>();
this.shouldCheckRelationAuthentication = shouldCheckRelationAuthentication;
}
/**
@ -161,13 +159,7 @@ public class CascadesContext implements ScheduleContext {
public static CascadesContext initContext(StatementContext statementContext,
Plan initPlan, PhysicalProperties requireProperties) {
return newContext(Optional.empty(), Optional.empty(), statementContext,
initPlan, new CTEContext(), requireProperties, true);
}
public static CascadesContext initViewContext(StatementContext statementContext,
Plan initPlan, PhysicalProperties requireProperties) {
return newContext(Optional.empty(), Optional.empty(), statementContext,
initPlan, new CTEContext(), requireProperties, false);
initPlan, new CTEContext(), requireProperties);
}
/**
@ -176,14 +168,14 @@ public class CascadesContext implements ScheduleContext {
public static CascadesContext newContextWithCteContext(CascadesContext cascadesContext,
Plan initPlan, CTEContext cteContext) {
return newContext(Optional.of(cascadesContext), Optional.empty(),
cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY,
cascadesContext.shouldCheckRelationAuthentication);
cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY
);
}
public static CascadesContext newCurrentTreeContext(CascadesContext context) {
return CascadesContext.newContext(context.getParent(), context.getCurrentTree(), context.getStatementContext(),
context.getRewritePlan(), context.getCteContext(),
context.getCurrentJobContext().getRequiredProperties(), context.shouldCheckRelationAuthentication);
context.getCurrentJobContext().getRequiredProperties());
}
/**
@ -192,14 +184,14 @@ public class CascadesContext implements ScheduleContext {
public static CascadesContext newSubtreeContext(Optional<CTEId> subtree, CascadesContext context,
Plan plan, PhysicalProperties requireProperties) {
return CascadesContext.newContext(Optional.of(context), subtree, context.getStatementContext(),
plan, context.getCteContext(), requireProperties, context.shouldCheckRelationAuthentication);
plan, context.getCteContext(), requireProperties);
}
private static CascadesContext newContext(Optional<CascadesContext> parent, Optional<CTEId> subtree,
StatementContext statementContext, Plan initPlan, CTEContext cteContext,
PhysicalProperties requireProperties, boolean shouldCheckRelationAuthentication) {
PhysicalProperties requireProperties) {
return new CascadesContext(parent, subtree, statementContext, initPlan, null,
cteContext, requireProperties, shouldCheckRelationAuthentication);
cteContext, requireProperties);
}
public CascadesContext getRoot() {
@ -655,10 +647,6 @@ public class CascadesContext implements ScheduleContext {
isLeadingJoin = leadingJoin;
}
public boolean shouldCheckRelationAuthentication() {
return shouldCheckRelationAuthentication;
}
public boolean isLeadingDisableJoinReorder() {
return isLeadingDisableJoinReorder;
}
@ -675,6 +663,10 @@ public class CascadesContext implements ScheduleContext {
planProcesses.add(planProcess);
}
public void addPlanProcesses(List<PlanProcess> planProcesses) {
this.planProcesses.addAll(planProcesses);
}
public List<PlanProcess> getPlanProcesses() {
return planProcesses;
}
@ -706,4 +698,13 @@ public class CascadesContext implements ScheduleContext {
}
}
}
/** keepOrShowPlanProcess */
public void keepOrShowPlanProcess(boolean showPlanProcess, Runnable task) {
if (showPlanProcess) {
withPlanProcess(showPlanProcess, task);
} else {
task.run();
}
}
}

View File

@ -42,6 +42,7 @@ import org.apache.doris.nereids.rules.rewrite.CheckAndStandardizeWindowFunctionA
import org.apache.doris.nereids.rules.rewrite.CheckDataTypes;
import org.apache.doris.nereids.rules.rewrite.CheckMatchExpression;
import org.apache.doris.nereids.rules.rewrite.CheckMultiDistinct;
import org.apache.doris.nereids.rules.rewrite.CheckPrivileges;
import org.apache.doris.nereids.rules.rewrite.CollectFilterAboveConsumer;
import org.apache.doris.nereids.rules.rewrite.CollectProjectAboveConsumer;
import org.apache.doris.nereids.rules.rewrite.ColumnPruning;
@ -66,7 +67,7 @@ import org.apache.doris.nereids.rules.rewrite.EliminateNullAwareLeftAntiJoin;
import org.apache.doris.nereids.rules.rewrite.EliminateOrderByConstant;
import org.apache.doris.nereids.rules.rewrite.EliminateSemiJoin;
import org.apache.doris.nereids.rules.rewrite.EliminateSort;
import org.apache.doris.nereids.rules.rewrite.EliminateSortUnderSubquery;
import org.apache.doris.nereids.rules.rewrite.EliminateSortUnderSubqueryOrView;
import org.apache.doris.nereids.rules.rewrite.EliminateUnnecessaryProject;
import org.apache.doris.nereids.rules.rewrite.EnsureProjectOnTopJoin;
import org.apache.doris.nereids.rules.rewrite.ExtractAndNormalizeWindowExpression;
@ -78,6 +79,7 @@ import org.apache.doris.nereids.rules.rewrite.InferFilterNotNull;
import org.apache.doris.nereids.rules.rewrite.InferJoinNotNull;
import org.apache.doris.nereids.rules.rewrite.InferPredicates;
import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct;
import org.apache.doris.nereids.rules.rewrite.InlineLogicalView;
import org.apache.doris.nereids.rules.rewrite.LimitSortToTopN;
import org.apache.doris.nereids.rules.rewrite.MergeFilters;
import org.apache.doris.nereids.rules.rewrite.MergeOneRowRelationIntoUnion;
@ -142,7 +144,7 @@ public class Rewriter extends AbstractBatchJobExecutor {
topic("Plan Normalization",
topDown(
new EliminateOrderByConstant(),
new EliminateSortUnderSubquery(),
new EliminateSortUnderSubqueryOrView(),
new EliminateGroupByConstant(),
// MergeProjects depends on this rule
new LogicalSubQueryAliasToLogicalProject(),
@ -231,6 +233,17 @@ public class Rewriter extends AbstractBatchJobExecutor {
new ApplyToJoin()
)
),
// before `Subquery unnesting` topic, some correlate slots should have appeared at LogicalApply.left,
// but it appeared at LogicalApply.right. After the `Subquery unnesting` topic, all slots is placed in a
// normal position, then we can check column privileges by these steps
//
// 1. use ColumnPruning rule to derive the used slots in LogicalView
// 2. and then check the column privileges
// 3. finally, we can eliminate the LogicalView
topic("Inline view and check column privileges",
custom(RuleType.CHECK_PRIVILEGES, CheckPrivileges::new),
bottomUp(new InlineLogicalView())
),
topic("Eliminate optimization",
bottomUp(
new EliminateLimit(),

View File

@ -107,7 +107,8 @@ public enum RuleType {
ELIMINATE_LOGICAL_SELECT_HINT(RuleTypeClass.REWRITE),
ELIMINATE_ORDER_BY_CONSTANT(RuleTypeClass.REWRITE),
ELIMINATE_SUBQUERY_ORDER_BY(RuleTypeClass.REWRITE),
ELIMINATE_ORDER_BY_UNDER_SUBQUERY(RuleTypeClass.REWRITE),
ELIMINATE_ORDER_BY_UNDER_VIEW(RuleTypeClass.REWRITE),
ELIMINATE_HINT(RuleTypeClass.REWRITE),
ELIMINATE_JOIN_ON_EMPTYRELATION(RuleTypeClass.REWRITE),
ELIMINATE_FILTER_ON_EMPTYRELATION(RuleTypeClass.REWRITE),
@ -288,6 +289,8 @@ public enum RuleType {
CTE_INLINE(RuleTypeClass.REWRITE),
REWRITE_CTE_CHILDREN(RuleTypeClass.REWRITE),
COLLECT_FILTER_ON_CONSUMER(RuleTypeClass.REWRITE),
INLINE_VIEW(RuleTypeClass.REWRITE),
CHECK_PRIVILEGES(RuleTypeClass.REWRITE),
COLLECT_FILTER(RuleTypeClass.REWRITE),
COLLECT_JOIN_CONSTRAINT(RuleTypeClass.REWRITE),

View File

@ -59,6 +59,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
import org.apache.doris.nereids.trees.plans.logical.LogicalTestScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalView;
import org.apache.doris.nereids.util.RelationUtil;
import org.apache.doris.qe.ConnectContext;
@ -157,7 +159,7 @@ public class BindRelation extends OneAnalysisRuleFactory {
}
// TODO: should generate different Scan sub class according to table's type
LogicalPlan scan = getAndCheckLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
LogicalPlan scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
if (cascadesContext.isLeadingJoin()) {
LeadingHint leading = (LeadingHint) cascadesContext.getHintMap().get("Leading");
leading.putRelationIdAndTableName(Pair.of(unboundRelation.getRelationId(), tableName));
@ -178,7 +180,7 @@ public class BindRelation extends OneAnalysisRuleFactory {
if (table == null) {
table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv());
}
return getAndCheckLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
}
private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, List<String> tableQualifier) {
@ -234,25 +236,18 @@ public class BindRelation extends OneAnalysisRuleFactory {
return scan;
}
private LogicalPlan getAndCheckLogicalPlan(TableIf table, UnboundRelation unboundRelation,
private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation,
List<String> tableQualifier, CascadesContext cascadesContext) {
// if current context is in the view, we can skip check authentication because
// the view already checked authentication
if (cascadesContext.shouldCheckRelationAuthentication()) {
UserAuthentication.checkPermission(table, cascadesContext.getConnectContext());
}
return doGetLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext);
}
private LogicalPlan doGetLogicalPlan(TableIf table, UnboundRelation unboundRelation, List<String> tableQualifier,
CascadesContext cascadesContext) {
switch (table.getType()) {
case OLAP:
case MATERIALIZED_VIEW:
return makeOlapScan(table, unboundRelation, tableQualifier);
case VIEW:
Plan viewPlan = parseAndAnalyzeView(((View) table).getInlineViewDef(), cascadesContext);
return new LogicalSubQueryAlias<>(tableQualifier, viewPlan);
View view = (View) table;
String inlineViewDef = view.getInlineViewDef();
Plan viewBody = parseAndAnalyzeView(inlineViewDef, cascadesContext);
LogicalView<Plan> logicalView = new LogicalView<>(view, viewBody);
return new LogicalSubQueryAlias<>(tableQualifier, logicalView);
case HMS_EXTERNAL_TABLE:
if (Config.enable_query_hive_views && ((HMSExternalTable) table).isView()) {
String hiveCatalog = ((HMSExternalTable) table).getCatalog().getName();
@ -276,6 +271,8 @@ public class BindRelation extends OneAnalysisRuleFactory {
return new LogicalOdbcScan(unboundRelation.getRelationId(), table, tableQualifier);
case ES_EXTERNAL_TABLE:
return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, tableQualifier);
case TEST_EXTERNAL_TABLE:
return new LogicalTestScan(unboundRelation.getRelationId(), table, tableQualifier);
default:
throw new AnalysisException("Unsupported tableType " + table.getType());
}
@ -299,9 +296,12 @@ public class BindRelation extends OneAnalysisRuleFactory {
if (parsedViewPlan instanceof UnboundResultSink) {
parsedViewPlan = (LogicalPlan) ((UnboundResultSink<?>) parsedViewPlan).child();
}
CascadesContext viewContext = CascadesContext.initViewContext(
CascadesContext viewContext = CascadesContext.initContext(
parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY);
viewContext.newAnalyzer(true, customTableResolver).analyze();
viewContext.keepOrShowPlanProcess(parentContext.showPlanProcess(), () -> {
viewContext.newAnalyzer(true, customTableResolver).analyze();
});
parentContext.addPlanProcesses(viewContext.getPlanProcesses());
// we should remove all group expression of the plan which in other memo, so the groupId would not conflict
return viewContext.getRewritePlan();
}

View File

@ -108,6 +108,7 @@ public class EliminateLogicalSelectHint extends OneRewriteRuleFactory {
}
throw new AnalysisException("The nereids is disabled in this sql, fallback to original planner");
}
context.invalidCache(selectHint.getHintName());
}
private void extractLeading(SelectHintLeading selectHint, CascadesContext context,

View File

@ -19,18 +19,20 @@ package org.apache.doris.nereids.rules.analysis;
import org.apache.doris.catalog.DatabaseIf;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.UserException;
import org.apache.doris.datasource.CatalogIf;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.qe.ConnectContext;
import java.util.Set;
/**
* Check whether a user is permitted to scan specific tables.
*/
public class UserAuthentication {
/** checkPermission. */
public static void checkPermission(TableIf table, ConnectContext connectContext) {
public static void checkPermission(TableIf table, ConnectContext connectContext, Set<String> columns)
throws UserException {
if (table == null) {
return;
}
@ -40,7 +42,7 @@ public class UserAuthentication {
}
String tableName = table.getName();
DatabaseIf db = table.getDatabase();
// when table inatanceof FunctionGenTable,db will be null
// when table instanceof FunctionGenTable,db will be null
if (db == null) {
return;
}
@ -50,13 +52,7 @@ public class UserAuthentication {
return;
}
String ctlName = catalog.getName();
// TODO: 2023/7/19 checkColumnsPriv
if (!connectContext.getEnv().getAccessManager().checkTblPriv(connectContext, ctlName, dbName,
tableName, PrivPredicate.SELECT)) {
String message = ErrorCode.ERR_TABLEACCESS_DENIED_ERROR.formatErrorMsg("SELECT",
ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(),
ctlName + ": " + dbName + ": " + tableName);
throw new AnalysisException(message);
}
connectContext.getEnv().getAccessManager().checkColumnsPriv(
connectContext.getCurrentUserIdentity(), ctlName, dbName, tableName, columns, PrivPredicate.SELECT);
}
}

View File

@ -0,0 +1,87 @@
// 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.rules.rewrite;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.common.UserException;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.jobs.JobContext;
import org.apache.doris.nereids.rules.analysis.UserAuthentication;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation;
import org.apache.doris.nereids.trees.plans.logical.LogicalRelation;
import org.apache.doris.nereids.trees.plans.logical.LogicalView;
import org.apache.doris.qe.ConnectContext;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/** CheckPrivileges */
public class CheckPrivileges extends ColumnPruning {
private JobContext jobContext;
@Override
public Plan rewriteRoot(Plan plan, JobContext jobContext) {
this.jobContext = jobContext;
super.rewriteRoot(plan, jobContext);
// don't rewrite plan
return plan;
}
@Override
public Plan visitLogicalView(LogicalView<? extends Plan> view, PruneContext context) {
checkColumnPrivileges(view.getView(), computeUsedColumns(view, context.requiredSlots));
// stop check privilege in the view
return view;
}
@Override
public Plan visitLogicalRelation(LogicalRelation relation, PruneContext context) {
if (relation instanceof LogicalCatalogRelation) {
TableIf table = ((LogicalCatalogRelation) relation).getTable();
checkColumnPrivileges(table, computeUsedColumns(relation, context.requiredSlots));
}
return super.visitLogicalRelation(relation, context);
}
private Set<String> computeUsedColumns(Plan plan, Set<Slot> requiredSlots) {
Map<Integer, Slot> idToSlot = plan.getOutputSet()
.stream()
.collect(Collectors.toMap(slot -> slot.getExprId().asInt(), slot -> slot));
return requiredSlots
.stream()
.map(slot -> idToSlot.get(slot.getExprId().asInt()))
.filter(slot -> slot != null)
.map(NamedExpression::getName)
.collect(Collectors.toSet());
}
private void checkColumnPrivileges(TableIf table, Set<String> usedColumns) {
ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext();
try {
UserAuthentication.checkPermission(table, connectContext, usedColumns);
} catch (UserException e) {
throw new AnalysisException(e.getMessage(), e);
}
}
}

View File

@ -0,0 +1,44 @@
// 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.rules.rewrite;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* SELECT * FROM lineorder ORDER BY 'f' -> SELECT * FROM lineorder
*/
public class EliminateSortUnderSubqueryOrView implements RewriteRuleFactory {
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
RuleType.ELIMINATE_ORDER_BY_UNDER_SUBQUERY.build(
logicalSubQueryAlias(logicalSort())
.then(subq -> subq.withChildren(subq.child().child(0)))
),
RuleType.ELIMINATE_ORDER_BY_UNDER_VIEW.build(
logicalView(logicalSort())
.then(view -> view.withChildren(view.child().child(0)))
)
);
}
}

View File

@ -19,15 +19,12 @@ package org.apache.doris.nereids.rules.rewrite;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.logical.LogicalView;
/**
* SELECT * FROM lineorder ORDER BY 'f' -> SELECT * FROM lineorder
*/
public class EliminateSortUnderSubquery extends OneRewriteRuleFactory {
/** InlineLogicalView */
public class InlineLogicalView extends OneRewriteRuleFactory {
@Override
public Rule build() {
return logicalSubQueryAlias(logicalSort())
.then(subq -> subq.withChildren(subq.child().child(0)))
.toRule(RuleType.ELIMINATE_SUBQUERY_ORDER_BY);
return logicalView().then(LogicalView::child).toRule(RuleType.INLINE_VIEW);
}
}

View File

@ -35,6 +35,7 @@ public enum PlanType {
LOGICAL_JDBC_SCAN,
LOGICAL_ODBC_SCAN,
LOGICAL_OLAP_SCAN,
LOGICAL_TEST_SCAN,
LOGICAL_ONE_ROW_RELATION,
LOGICAL_SCHEMA_SCAN,
LOGICAL_TVF_RELATION,
@ -71,6 +72,7 @@ public enum PlanType {
LOGICAL_REPEAT,
LOGICAL_SELECT_HINT,
LOGICAL_SUBQUERY_ALIAS,
LOGICAL_VIEW,
LOGICAL_SORT,
LOGICAL_TOP_N,
LOGICAL_UNION,

View File

@ -0,0 +1,71 @@
// 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.trees.plans.logical;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.RelationId;
import org.apache.doris.nereids.util.Utils;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
/**
* LogicalTestScan.
*
* only for ut
*/
public class LogicalTestScan extends LogicalCatalogRelation {
public LogicalTestScan(RelationId relationId, TableIf table, List<String> qualifier) {
this(relationId, table, qualifier, Optional.empty(), Optional.empty());
}
private LogicalTestScan(RelationId relationId, TableIf table, List<String> qualifier,
Optional<GroupExpression> groupExpression, Optional<LogicalProperties> logicalProperties) {
super(relationId, PlanType.LOGICAL_TEST_SCAN, table, qualifier, groupExpression, logicalProperties);
}
@Override
public String toString() {
return Utils.toSqlString("LogicalTestScan",
"qualified", qualifiedName(),
"output", getOutput()
);
}
@Override
public boolean equals(Object o) {
return super.equals(o) && Objects.equals(table, ((LogicalTestScan) table).getTable());
}
@Override
public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
return new LogicalTestScan(relationId, table, qualifier,
groupExpression, Optional.ofNullable(getLogicalProperties()));
}
@Override
public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, List<Plan> children) {
return new LogicalTestScan(relationId, table, qualifier, groupExpression, logicalProperties);
}
}

View File

@ -0,0 +1,145 @@
// 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.trees.plans.logical;
import org.apache.doris.catalog.View;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.properties.FdItem;
import org.apache.doris.nereids.properties.FunctionalDependencies;
import org.apache.doris.nereids.properties.LogicalProperties;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.Utils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.function.Supplier;
/** LogicalView */
public class LogicalView<BODY extends Plan> extends LogicalUnary<BODY> {
private final View view;
/** LogicalView */
public LogicalView(View view, BODY body) {
super(PlanType.LOGICAL_VIEW, Optional.empty(), Optional.empty(), body);
this.view = Objects.requireNonNull(view, "catalog can not be null");
if (!(body instanceof LogicalPlan)) {
throw new AnalysisException("Child of LogicalView should be LogicalPlan, but meet: " + body.getClass());
}
}
@Override
public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
return visitor.visitLogicalView(this, context);
}
@Override
public List<? extends Expression> getExpressions() {
return ImmutableList.of();
}
public String getCatalog() {
return view.getDatabase().getCatalog().getName();
}
public String getDb() {
return view.getDatabase().getFullName();
}
public String getName() {
return view.getName();
}
public String getViewString() {
return view.getInlineViewDef();
}
public View getView() {
return view;
}
@Override
public LogicalProperties getLogicalProperties() {
return child().getLogicalProperties();
}
@Override
public Plan withGroupExpression(Optional<GroupExpression> groupExpression) {
return new LogicalView(view, child());
}
@Override
public Plan withGroupExprLogicalPropChildren(Optional<GroupExpression> groupExpression,
Optional<LogicalProperties> logicalProperties, List<Plan> children) {
return new LogicalView(view, child());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LogicalView that = (LogicalView) o;
return Objects.equals(view, that.view);
}
@Override
public String toString() {
return Utils.toSqlString("LogicalView",
"catalog", getCatalog(),
"db", getDb(),
"name", getName()
);
}
@Override
public int hashCode() {
return Objects.hash(getCatalog(), getDb(), getName());
}
@Override
public List<Slot> computeOutput() {
return child().getOutput();
}
@Override
public FunctionalDependencies computeFuncDeps(Supplier<List<Slot>> outputSupplier) {
return ((LogicalPlan) child()).computeFuncDeps(outputSupplier);
}
@Override
public ImmutableSet<FdItem> computeFdItems(Supplier<List<Slot>> outputSupplier) {
return ((LogicalPlan) child()).computeFdItems(outputSupplier);
}
@Override
public Plan withChildren(List<Plan> children) {
return new LogicalView<>(view, (LogicalPlan) children.get(0));
}
}

View File

@ -48,6 +48,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias;
import org.apache.doris.nereids.trees.plans.logical.LogicalTopN;
import org.apache.doris.nereids.trees.plans.logical.LogicalUnion;
import org.apache.doris.nereids.trees.plans.logical.LogicalView;
import org.apache.doris.nereids.trees.plans.logical.LogicalWindow;
import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin;
import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort;
@ -228,6 +229,10 @@ public abstract class PlanVisitor<R, C> implements CommandVisitor<R, C>, Relatio
return visit(alias, context);
}
public R visitLogicalView(LogicalView<? extends Plan> alias, C context) {
return visit(alias, context);
}
public R visitLogicalTopN(LogicalTopN<? extends Plan> topN, C context) {
return visit(topN, context);
}

View File

@ -2633,8 +2633,12 @@ public class SessionVariable implements Serializable, Writable {
}
public Set<String> getDisableNereidsRuleNames() {
String checkPrivilege = RuleType.CHECK_PRIVILEGES.name();
String checkRowPolicy = RuleType.CHECK_ROW_POLICY.name();
return Arrays.stream(disableNereidsRules.split(",[\\s]*"))
.map(rule -> rule.toUpperCase(Locale.ROOT))
.filter(rule -> !StringUtils.equalsIgnoreCase(rule, checkPrivilege)
&& !StringUtils.equalsIgnoreCase(rule, checkRowPolicy))
.collect(ImmutableSet.toImmutableSet());
}
@ -2642,7 +2646,10 @@ public class SessionVariable implements Serializable, Writable {
return Arrays.stream(disableNereidsRules.split(",[\\s]*"))
.filter(rule -> !rule.isEmpty())
.map(rule -> rule.toUpperCase(Locale.ROOT))
.map(rule -> RuleType.valueOf(rule).type())
.map(rule -> RuleType.valueOf(rule))
.filter(ruleType -> ruleType != RuleType.CHECK_PRIVILEGES
&& ruleType != RuleType.CHECK_ROW_POLICY)
.map(RuleType::type)
.collect(ImmutableSet.toImmutableSet());
}

View File

@ -0,0 +1,390 @@
// 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.privileges;
import org.apache.doris.analysis.UserIdentity;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.PrimitiveType;
import org.apache.doris.common.AuthorizationException;
import org.apache.doris.common.FeConstants;
import org.apache.doris.datasource.CatalogMgr;
import org.apache.doris.datasource.test.TestExternalCatalog.TestCatalogProvider;
import org.apache.doris.mysql.privilege.AccessControllerFactory;
import org.apache.doris.mysql.privilege.AccessControllerManager;
import org.apache.doris.mysql.privilege.CatalogAccessController;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.util.PlanChecker;
import org.apache.doris.utframe.TestWithFeService;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import mockit.Expectations;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class TestCheckPrivileges extends TestWithFeService {
private static final Map<String, Map<String, List<Column>>> CATALOG_META = ImmutableMap.of(
"test_db", ImmutableMap.of(
"test_tbl1", ImmutableList.of(
new Column("id", PrimitiveType.INT),
new Column("name", PrimitiveType.VARCHAR)
),
"test_tbl2", ImmutableList.of(
new Column("id", PrimitiveType.INT),
new Column("name", PrimitiveType.VARCHAR)
),
"test_tbl3", ImmutableList.of(
new Column("id", PrimitiveType.INT),
new Column("name", PrimitiveType.VARCHAR)
)
)
);
@Test
public void testColumnPrivileges() throws Exception {
FeConstants.runningUnitTest = true;
String catalogProvider
= "org.apache.doris.nereids.privileges.TestCheckPrivileges$CustomCatalogProvider";
String accessControllerFactory
= "org.apache.doris.nereids.privileges.TestCheckPrivileges$CustomAccessControllerFactory";
String catalog = "custom_catalog";
String db = "test_db";
createCatalog("create catalog " + catalog + " properties("
+ " \"type\"=\"test\","
+ " \"catalog_provider.class\"=\"" + catalogProvider + "\","
+ " \"" + CatalogMgr.ACCESS_CONTROLLER_CLASS_PROP + "\"=\"" + accessControllerFactory + "\""
+ ")");
createDatabase("internal_db");
String internalDb = "internal_db";
String table1 = "test_tbl1";
String table2 = "test_tbl2";
String table3 = "test_tbl3";
String view1 = "query_tbl2_view1";
createView("create view " + internalDb + "."
+ view1 + " as select * from custom_catalog.test_db." + table2);
String view2 = "query_tbl2_view2";
createView("create view " + internalDb + "."
+ view2 + " as select * from custom_catalog.test_db." + table2);
String view3 = "query_tbl2_view3";
createView("create view " + internalDb + "."
+ view3 + " as select * from custom_catalog.test_db." + table3);
String view4 = "query_tbl2_view4";
createView("create view " + internalDb + "."
+ view4 + " as select * from " + internalDb + "." + view3);
String user = "test_nereids_privilege_user";
addUser(user, true);
useUser(user);
List<MakeTablePrivileges> privileges = ImmutableList.of(
// base table privileges
MakePrivileges.table(catalog, db, table1).allowSelectTable(user),
MakePrivileges.table(catalog, db, table2).allowSelectColumns(user, ImmutableSet.of("id")),
// view privileges
MakePrivileges.table("internal", internalDb, view1).allowSelectTable(user),
MakePrivileges.table("internal", internalDb, view2)
.allowSelectColumns(user, ImmutableSet.of("name")),
MakePrivileges.table("internal", internalDb, view4)
.allowSelectColumns(user, ImmutableSet.of("id"))
);
AccessControllerManager accessManager = Env.getCurrentEnv().getAccessManager();
CatalogAccessController catalogAccessController = accessManager.getAccessControllerOrDefault(catalog);
new Expectations(accessManager) {
{
accessManager.getAccessControllerOrDefault("internal");
minTimes = 0;
result = catalogAccessController;
}
};
withPrivileges(privileges, () -> {
// test base table
{
// has table privilege
query("select * from custom_catalog.test_db.test_tbl1");
// has id column privilege
query("select id from custom_catalog.test_db.test_tbl2");
// no name column privilege, throw exception:
//
// Permission denied: user ['test_nereids_privilege_user'@'%'] does not have privilege for
// [priv predicate: OR, Admin_priv Select_priv ] command on
// [custom_catalog].[test_db].[test_tbl2].[name]
Assertions.assertThrows(AnalysisException.class, () ->
query("select * from custom_catalog.test_db.test_tbl2")
);
// no table privilege
Assertions.assertThrows(AnalysisException.class, () ->
query("select * from custom_catalog.test_db.test_tbl3")
);
}
// test view
{
// has view privilege
query("select * from " + internalDb + "." + view1);
// has view name privilege
query("select name from " + internalDb + "." + view2);
// no id column privilege
Assertions.assertThrows(AnalysisException.class, () ->
query("select id from " + internalDb + "." + view2)
);
// no view privilege
Assertions.assertThrows(AnalysisException.class, () ->
query("select * from " + internalDb + "." + view3)
);
// has id column privilege
query("select id from " + internalDb + "." + view4);
// no name column privilege
Assertions.assertThrows(AnalysisException.class, () ->
query("select name from " + internalDb + "." + view4)
);
}
});
}
private void query(String sql) {
PlanChecker.from(connectContext)
.parse(sql)
.analyze()
.rewrite();
}
private void withPrivileges(List<MakeTablePrivileges> privileges, Runnable task) {
List<TablePrivilege> tablePrivileges = Lists.newArrayList();
List<ColumnPrivilege> columnPrivileges = Lists.newArrayList();
for (MakeTablePrivileges privilege : privileges) {
tablePrivileges.addAll(privilege.tablePrivileges);
columnPrivileges.addAll(privilege.columnPrivileges);
}
SimpleCatalogAccessController.tablePrivileges.set(tablePrivileges);
SimpleCatalogAccessController.columnPrivileges.set(columnPrivileges);
try {
task.run();
} finally {
SimpleCatalogAccessController.tablePrivileges.remove();
SimpleCatalogAccessController.columnPrivileges.remove();
}
}
public static class CustomCatalogProvider implements TestCatalogProvider {
@Override
public Map<String, Map<String, List<Column>>> getMetadata() {
return CATALOG_META;
}
}
public static class CustomAccessControllerFactory implements AccessControllerFactory {
@Override
public CatalogAccessController createAccessController(Map<String, String> prop) {
return new SimpleCatalogAccessController();
}
}
public static class SimpleCatalogAccessController implements CatalogAccessController {
private static ThreadLocal<List<TablePrivilege>> tablePrivileges = new ThreadLocal<>();
private static ThreadLocal<List<ColumnPrivilege>> columnPrivileges = new ThreadLocal<>();
@Override
public boolean checkGlobalPriv(UserIdentity currentUser, PrivPredicate wanted) {
return true;
}
@Override
public boolean checkCtlPriv(UserIdentity currentUser, String ctl, PrivPredicate wanted) {
return true;
}
@Override
public boolean checkDbPriv(UserIdentity currentUser, String ctl, String db, PrivPredicate wanted) {
return true;
}
@Override
public boolean checkTblPriv(UserIdentity currentUser, String ctl, String db, String tbl, PrivPredicate wanted) {
List<TablePrivilege> tablePrivileges = SimpleCatalogAccessController.tablePrivileges.get();
if (!CollectionUtils.isEmpty(tablePrivileges)
&& tablePrivileges.stream().anyMatch(p -> p.checkTblPriv(currentUser, ctl, db, tbl))) {
return true;
}
List<ColumnPrivilege> columnPrivileges = SimpleCatalogAccessController.columnPrivileges.get();
if (!CollectionUtils.isEmpty(columnPrivileges)
&& columnPrivileges.stream().anyMatch(p -> p.checkTblPriv(currentUser, ctl, db, tbl))) {
return true;
}
return false;
}
@Override
public boolean checkResourcePriv(UserIdentity currentUser, String resourceName, PrivPredicate wanted) {
return true;
}
@Override
public boolean checkWorkloadGroupPriv(UserIdentity currentUser, String workloadGroupName,
PrivPredicate wanted) {
return true;
}
@Override
public void checkColsPriv(UserIdentity currentUser, String ctl, String db, String tbl, Set<String> cols,
PrivPredicate wanted) throws AuthorizationException {
List<TablePrivilege> tablePrivileges = SimpleCatalogAccessController.tablePrivileges.get();
if (!CollectionUtils.isEmpty(tablePrivileges)
&& tablePrivileges.stream().anyMatch(p -> p.checkTblPriv(currentUser, ctl, db, tbl))) {
return;
}
List<ColumnPrivilege> columnPrivileges = SimpleCatalogAccessController.columnPrivileges.get();
if (CollectionUtils.isEmpty(columnPrivileges)) {
String format = "Permission denied: user [%s] does not have privilege "
+ "for [%s] command on [%s].[%s].[%s].[%s]";
throw new AuthorizationException(String.format(
format,
currentUser, wanted, ctl, db, tbl, cols.iterator().next()));
}
for (String col : cols) {
boolean hasPrivilege = columnPrivileges.stream()
.anyMatch(t -> t.checkColsPriv(currentUser, ctl, db, tbl, col));
if (!hasPrivilege) {
String format = "Permission denied: user [%s] does not have privilege "
+ "for [%s] command on [%s].[%s].[%s].[%s]";
throw new AuthorizationException(String.format(
format,
currentUser, wanted, ctl, db, tbl, col));
}
}
}
}
private static class MakePrivileges {
public static MakeTablePrivileges table(String catalog, String db, String table) {
return new MakeTablePrivileges(catalog, db, table);
}
}
private static class MakeTablePrivileges {
private String catalog;
private String db;
private String table;
private List<TablePrivilege> tablePrivileges;
private List<ColumnPrivilege> columnPrivileges;
public MakeTablePrivileges(String catalog, String db, String table) {
this.catalog = catalog;
this.db = db;
this.table = table;
this.tablePrivileges = Lists.newArrayList();
this.columnPrivileges = Lists.newArrayList();
}
public MakeTablePrivileges allowSelectTable(String user) {
tablePrivileges.add(new TablePrivilege(catalog, db, table, user));
return this;
}
public MakeTablePrivileges allowSelectColumns(String user, Set<String> allowColumns) {
columnPrivileges.add(new ColumnPrivilege(catalog, db, table, user, allowColumns));
return this;
}
}
private static class TablePrivilege {
private final String catalog;
private final String db;
private final String table;
private final String user;
public TablePrivilege(String catalog, String db, String table, String user) {
this.catalog = catalog;
this.db = db;
this.table = table;
this.user = user;
}
public boolean checkTblPriv(UserIdentity currentUser, String ctl, String db, String tbl) {
return isSameTable(ctl, db, tbl) && StringUtils.equals(this.user, currentUser.getQualifiedUser());
}
public boolean isSameTable(String catalog, String db, String tbl) {
return StringUtils.equals(this.catalog, catalog)
&& StringUtils.equals(this.db, db)
&& StringUtils.equals(this.table, tbl);
}
}
private static class ColumnPrivilege {
private final String catalog;
private final String db;
private final String table;
private final String user;
private final Set<String> allowColumns;
public ColumnPrivilege(String catalog, String db, String table, String user, Set<String> allowColumns) {
this.catalog = catalog;
this.db = db;
this.table = table;
this.user = user;
this.allowColumns = allowColumns;
}
public boolean checkTblPriv(UserIdentity currentUser, String ctl, String db, String tbl) {
return isSameTable(ctl, db, tbl) && StringUtils.equals(this.user, currentUser.getQualifiedUser());
}
public boolean checkColsPriv(UserIdentity currentUser, String ctl, String db, String tbl, String col) {
return isSameTable(ctl, db, tbl)
&& StringUtils.equals(this.user, currentUser.getQualifiedUser()) && allowColumns.contains(col);
}
public boolean isSameTable(String catalog, String db, String tbl) {
return StringUtils.equals(this.catalog, catalog)
&& StringUtils.equals(this.db, db)
&& StringUtils.equals(this.table, tbl);
}
}
}

View File

@ -24,6 +24,7 @@ import org.apache.doris.nereids.glue.translator.PlanTranslatorContext;
import org.apache.doris.nereids.parser.NereidsParser;
import org.apache.doris.nereids.properties.PhysicalProperties;
import org.apache.doris.nereids.rules.analysis.LogicalSubQueryAliasToLogicalProject;
import org.apache.doris.nereids.rules.rewrite.InlineLogicalView;
import org.apache.doris.nereids.rules.rewrite.MergeProjects;
import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
import org.apache.doris.nereids.util.MemoPatternMatchSupported;
@ -141,6 +142,7 @@ public class ViewTest extends TestWithFeService implements MemoPatternMatchSuppo
+ "ON X.ID1 = Y.ID3"
)
.applyTopDown(new LogicalSubQueryAliasToLogicalProject())
.applyBottomUp(new InlineLogicalView())
.applyTopDown(new MergeProjects())
.matches(
logicalProject(
@ -148,10 +150,10 @@ public class ViewTest extends TestWithFeService implements MemoPatternMatchSuppo
logicalProject(
logicalJoin(
logicalProject(
logicalOlapScan()
logicalOlapScan()
),
logicalProject(
logicalOlapScan()
logicalOlapScan()
)
)
),

View File

@ -102,7 +102,7 @@ import java.util.List;
public class OlapQueryCacheTest {
private static final Logger LOG = LogManager.getLogger(OlapQueryCacheTest.class);
public static String fullDbName = "testDb";
public static String userName = "testUser";
public static String userName = "root";
private static ConnectContext context;
@ -256,7 +256,7 @@ public class OlapQueryCacheTest {
ctx.getCurrentUserIdentity();
minTimes = 0;
UserIdentity userIdentity = new UserIdentity(userName, "192.168.1.1");
UserIdentity userIdentity = new UserIdentity(userName, "%");
userIdentity.setIsAnalyzed();
result = userIdentity;
}

View File

@ -29,6 +29,7 @@ import org.apache.doris.analysis.CreatePolicyStmt;
import org.apache.doris.analysis.CreateSqlBlockRuleStmt;
import org.apache.doris.analysis.CreateTableAsSelectStmt;
import org.apache.doris.analysis.CreateTableStmt;
import org.apache.doris.analysis.CreateUserStmt;
import org.apache.doris.analysis.CreateViewStmt;
import org.apache.doris.analysis.DropDbStmt;
import org.apache.doris.analysis.DropPolicyStmt;
@ -70,6 +71,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.util.MemoTestUtils;
import org.apache.doris.planner.Planner;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.DdlExecutor;
import org.apache.doris.qe.OriginStatement;
import org.apache.doris.qe.QueryState;
import org.apache.doris.qe.SessionVariable;
@ -754,12 +756,22 @@ public abstract class TestWithFeService {
}
protected void useUser(String userName) throws AnalysisException {
UserIdentity user = new UserIdentity(userName, "%");
useUser(userName, "%");
}
protected void useUser(String userName, String host) throws AnalysisException {
UserIdentity user = new UserIdentity(userName, host);
user.analyze();
connectContext.setCurrentUserIdentity(user);
connectContext.setQualifiedUser(userName);
}
protected void addUser(String userName, boolean ifNotExists) throws Exception {
CreateUserStmt createUserStmt = (CreateUserStmt) UtFrameUtils.parseAndAnalyzeStmt(
"create user " + (ifNotExists ? "if not exists " : "") + userName + "@'%'", connectContext);
DdlExecutor.execute(Env.getCurrentEnv(), createUserStmt);
}
protected void addRollup(String sql) throws Exception {
AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
Env.getCurrentEnv().alterTable(alterTableStmt);

View File

@ -37,7 +37,7 @@ suite("test_nereids_row_policy") {
sql "set enable_fallback_to_original_planner = false"
test {
sql "SELECT * FROM ${viewName}"
exception "SELECT command denied to user"
exception "does not have privilege for"
}
}
assertEquals(size, result1.size())

View File

@ -57,7 +57,7 @@ suite("view_authorization") {
// no privilege to base table
test {
sql "select * from ${db}.${baseTable}"
exception "SELECT command denied to user"
exception "does not have privilege for"
}
// has privilege to view1
@ -69,7 +69,7 @@ suite("view_authorization") {
// no privilege to view2
test {
sql "select * from ${db}.${view2}"
exception "SELECT command denied to user"
exception "does not have privilege for"
}
// nested view