Skip to content

Commit

Permalink
[Enhancement] (nereids)implement showPartitionIdCommand in nereids
Browse files Browse the repository at this point in the history
  • Loading branch information
Vallishp committed Nov 15, 2024
1 parent 608b0a8 commit 2224641
Show file tree
Hide file tree
Showing 6 changed files with 196 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,7 @@ supportedShowStatement
((FROM | IN) database=identifier)? #showView
| SHOW REPOSITORIES #showRepositories
| SHOW ROLES #showRoles
| SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId
| SHOW PROC path=STRING_LITERAL #showProc
| SHOW CREATE MATERIALIZED VIEW mvName=identifier
ON tableName=multipartIdentifier #showCreateMaterializedView
Expand Down Expand Up @@ -290,7 +291,6 @@ unsupportedShowStatement
| SHOW DATA (FROM tableName=multipartIdentifier)? sortClause? propertyClause? #showData
| SHOW TEMPORARY? PARTITIONS FROM tableName=multipartIdentifier
wildWhere? sortClause? limitClause? #showPartitions
| SHOW PARTITION partitionId=INTEGER_VALUE #showPartitionId
| SHOW TABLET tabletId=INTEGER_VALUE #showTabletId
| SHOW TABLETS BELONG
tabletIds+=INTEGER_VALUE (COMMA tabletIds+=INTEGER_VALUE)* #showTabletBelong
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@
import org.apache.doris.nereids.DorisParser.ShowCreateMaterializedViewContext;
import org.apache.doris.nereids.DorisParser.ShowCreateProcedureContext;
import org.apache.doris.nereids.DorisParser.ShowLastInsertContext;
import org.apache.doris.nereids.DorisParser.ShowPartitionIdContext;
import org.apache.doris.nereids.DorisParser.ShowProcContext;
import org.apache.doris.nereids.DorisParser.ShowProcedureStatusContext;
import org.apache.doris.nereids.DorisParser.ShowRepositoriesContext;
Expand Down Expand Up @@ -441,6 +442,7 @@
import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand;
Expand Down Expand Up @@ -4040,6 +4042,15 @@ public LogicalPlan visitShowLastInsert(ShowLastInsertContext ctx) {
return new ShowLastInsertCommand();
}

@Override
public LogicalPlan visitShowPartitionId(ShowPartitionIdContext ctx) {
long partitionId = -1;
if (ctx.partitionId != null) {
partitionId = Long.parseLong(ctx.partitionId.getText());
}
return new ShowPartitionIdCommand(partitionId);
}

@Override
public LogicalPlan visitShowVariables(ShowVariablesContext ctx) {
SetType type = SetType.DEFAULT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@ public enum PlanType {
SHOW_CONFIG_COMMAND,
SHOW_CREATE_MATERIALIZED_VIEW_COMMAND,
SHOW_LAST_INSERT_COMMAND,
SHOW_PARTITIONID_COMMAND,
SHOW_PROC_COMMAND,
SHOW_REPOSITORIES_COMMAND,
SHOW_ROLE_COMMAND,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,120 @@
// 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.commands;

import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Database;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.Partition;
import org.apache.doris.catalog.ScalarType;
import org.apache.doris.catalog.Table;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.nereids.trees.plans.PlanType;
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.ShowResultSet;
import org.apache.doris.qe.ShowResultSetMetaData;
import org.apache.doris.qe.StmtExecutor;

import com.google.common.collect.Lists;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;

import java.util.ArrayList;
import java.util.List;

/**
* show partition command
*/
public class ShowPartitionIdCommand extends ShowCommand {
public static final Logger LOG = LogManager.getLogger(ShowPartitionIdCommand.class);
private final long partitionId;

/**
* constructor
*/
public ShowPartitionIdCommand(long partitionId) {
super(PlanType.SHOW_PARTITIONID_COMMAND);
this.partitionId = partitionId;
}

/**
* get meta for show partionId
*/
public ShowResultSetMetaData getMetaData() {
ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder();
builder.addColumn(new Column("DbName", ScalarType.createVarchar(30)));
builder.addColumn(new Column("TableName", ScalarType.createVarchar(30)));
builder.addColumn(new Column("PartitionName", ScalarType.createVarchar(30)));
builder.addColumn(new Column("DbId", ScalarType.createVarchar(30)));
builder.addColumn(new Column("TableId", ScalarType.createVarchar(30)));
return builder.build();
}

private ShowResultSet handleShowPartitionId(ConnectContext ctx, StmtExecutor executor) throws Exception {
List<List<String>> rows = Lists.newArrayList();
Env env = ctx.getEnv();
List<Long> dbIds = env.getInternalCatalog().getDbIds();
for (long dbId : dbIds) {
Database database = env.getInternalCatalog().getDbNullable(dbId);
if (database == null) {
continue;
}
List<Table> tables = database.getTables();
for (Table tbl : tables) {
if (tbl instanceof OlapTable) {
tbl.readLock();
try {
Partition partition = ((OlapTable) tbl).getPartition(partitionId);
if (partition != null) {
List<String> row = new ArrayList<>();
row.add(database.getFullName());
row.add(tbl.getName());
row.add(partition.getName());
row.add(String.valueOf(database.getId()));
row.add(String.valueOf(tbl.getId()));
rows.add(row);
break;
}
} finally {
tbl.readUnlock();
}
}
}
}
ShowResultSet resultSet = new ShowResultSet(getMetaData(), rows);
return resultSet;
}

@Override
public ShowResultSet doRun(ConnectContext ctx, StmtExecutor executor) throws Exception {
// check access first
if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "SHOW PARTITION");
}
return handleShowPartitionId(ctx, executor);
}

@Override
public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
return visitor.visitShowPartitionIdCommand(this, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import org.apache.doris.nereids.trees.plans.commands.ShowCreateMaterializedViewCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowCreateProcedureCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowLastInsertCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowPartitionIdCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowProcCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowProcedureStatusCommand;
import org.apache.doris.nereids.trees.plans.commands.ShowRepositoriesCommand;
Expand Down Expand Up @@ -252,6 +253,10 @@ default R visitShowLastInsertCommand(ShowLastInsertCommand showLastInsertCommand
return visitCommand(showLastInsertCommand, context);
}

default R visitShowPartitionIdCommand(ShowPartitionIdCommand showPartitionIdCommand, C context) {
return visitCommand(showPartitionIdCommand, context);
}

default R visitShowVariablesCommand(ShowVariablesCommand showVariablesCommand, C context) {
return visitCommand(showVariablesCommand, context);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.


suite("test_nereids_showpartitionid") {
def table = "test_nereids_showpartitionid"
// create table and insert data
sql """ drop table if exists ${table} force"""
sql """
create table ${table} (
`id` int(11),
`name` varchar(128),
`da` date
)
engine=olap
duplicate key(id)
partition by range(da)(
PARTITION p3 VALUES LESS THAN ('2023-01-01'),
PARTITION p4 VALUES LESS THAN ('2024-01-01'),
PARTITION p5 VALUES LESS THAN ('2025-01-01')
)
distributed by hash(id) buckets 2
properties(
"replication_num"="1",
"light_schema_change"="true"
);
"""

result = sql_return_maparray "show partitions from ${table}"
logger.info("${result}")
def partitionId;
for (def partition : result) {
//get any partition ID.
partitionId = partition.PartitionId;
break;
}

checkNereidsExecute("show partition ${partitionId}")
def result = sql "show partition ${partitionId}";
logger.info("${result}");


}

0 comments on commit 2224641

Please sign in to comment.