[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:
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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(),
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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();
|
||||
}
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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)))
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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,
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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));
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
|
||||
@ -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());
|
||||
}
|
||||
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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()
|
||||
)
|
||||
)
|
||||
),
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -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
|
||||
|
||||
Reference in New Issue
Block a user