You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by tu...@apache.org on 2022/04/08 03:15:45 UTC

[shardingsphere] branch master updated: Support postgresql alter/drop index statement with qualified name (#16651)

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

tuichenchuxin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 7b5e711f3aa Support postgresql alter/drop index statement with qualified name (#16651)
7b5e711f3aa is described below

commit 7b5e711f3aa099d8e1e4177fea763ac422ec32e6
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri Apr 8 11:15:35 2022 +0800

    Support postgresql alter/drop index statement with qualified name (#16651)
---
 .../ShardingTableBroadcastRoutingEngine.java       | 22 ++-----------
 .../metadata/schema/util/IndexMetaDataUtil.java    | 28 ++++++++++++++++
 .../type/DropIndexStatementSchemaRefresher.java    | 37 +++++++---------------
 .../singletable/route/SingleTableSQLRouter.java    | 31 ++++++++----------
 .../main/antlr4/imports/opengauss/DDLStatement.g4  |  4 +--
 .../main/antlr4/imports/opengauss/DMLStatement.g4  |  4 ---
 .../impl/OpenGaussDDLStatementSQLVisitor.java      | 18 +++++++++--
 .../main/antlr4/imports/postgresql/DDLStatement.g4 |  4 +--
 .../main/antlr4/imports/postgresql/DMLStatement.g4 |  4 ---
 .../impl/PostgreSQLDDLStatementSQLVisitor.java     | 18 +++++++++--
 .../src/main/resources/case/ddl/alter-index.xml    |  1 +
 .../src/main/resources/case/ddl/drop-index.xml     |  1 +
 .../resources/sql/supported/ddl/alter-index.xml    |  1 +
 .../resources/sql/supported/ddl/drop-index.xml     |  1 +
 14 files changed, 95 insertions(+), 79 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
index c3c905714f6..1188dd337b1 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/broadcast/ShardingTableBroadcastRoutingEngine.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.IndexAvailable;
 import org.apache.shardingsphere.infra.datanode.DataNode;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.util.IndexMetaDataUtil;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
@@ -29,12 +30,10 @@ import org.apache.shardingsphere.sharding.route.engine.type.ShardingRouteEngine;
 import org.apache.shardingsphere.sharding.route.engine.type.complex.ShardingCartesianRoutingEngine;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
-import java.util.Optional;
 import java.util.stream.Collectors;
 
 /**
@@ -106,24 +105,7 @@ public final class ShardingTableBroadcastRoutingEngine implements ShardingRouteE
         if (!shardingRuleTableNames.isEmpty()) {
             return shardingRuleTableNames;
         }
-        return sqlStatementContext instanceof IndexAvailable ? getTableNamesFromMetaData(((IndexAvailable) sqlStatementContext).getIndexes()) : Collections.emptyList();
-    }
-    
-    private Collection<String> getTableNamesFromMetaData(final Collection<IndexSegment> indexes) {
-        Collection<String> result = new LinkedList<>();
-        for (IndexSegment each : indexes) {
-            findLogicTableNameFromMetaData(each.getIdentifier().getValue()).ifPresent(result::add);
-        }
-        return result;
-    }
-    
-    private Optional<String> findLogicTableNameFromMetaData(final String logicIndexName) {
-        for (String each : schema.getAllTableNames()) {
-            if (schema.get(each).getIndexes().containsKey(logicIndexName)) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
+        return sqlStatementContext instanceof IndexAvailable ? IndexMetaDataUtil.getTableNamesFromMetaData(schema, ((IndexAvailable) sqlStatementContext).getIndexes()) : Collections.emptyList();
     }
     
     private Collection<RouteUnit> getBroadcastTableRouteUnits(final ShardingRule shardingRule, final String broadcastTableName) {
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/util/IndexMetaDataUtil.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/util/IndexMetaDataUtil.java
index ef01743a6d3..160b3ae8dee 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/util/IndexMetaDataUtil.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/util/IndexMetaDataUtil.java
@@ -20,9 +20,13 @@ package org.apache.shardingsphere.infra.metadata.schema.util;
 import com.google.common.base.Strings;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.index.IndexSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 
 import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
 
 /**
  * Index meta data utility class.
@@ -70,4 +74,28 @@ public class IndexMetaDataUtil {
         }
         return builder.append(GENERATED_LOGIC_INDEX_NAME_SUFFIX).toString();
     }
+    
+    /**
+     * Get table names from metadata.
+     *
+     * @param schema schema
+     * @param indexes indexes
+     * @return table names
+     */
+    public static Collection<String> getTableNamesFromMetaData(final ShardingSphereSchema schema, final Collection<IndexSegment> indexes) {
+        Collection<String> result = new LinkedList<>();
+        for (IndexSegment each : indexes) {
+            findLogicTableNameFromMetaData(schema, each.getIdentifier().getValue()).ifPresent(result::add);
+        }
+        return result;
+    }
+    
+    private static Optional<String> findLogicTableNameFromMetaData(final ShardingSphereSchema schema, final String logicIndexName) {
+        for (String each : schema.getAllTableNames()) {
+            if (schema.get(each).getIndexes().containsKey(logicIndexName)) {
+                return Optional.of(each);
+            }
+        }
+        return Optional.empty();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
index 60efbbfbb19..1fc3ef84968 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/refresher/type/DropIndexStatementSchemaRefresher.java
@@ -17,8 +17,6 @@
 
 package org.apache.shardingsphere.infra.context.refresher.type;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.context.refresher.MetaDataRefresher;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
@@ -28,6 +26,7 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.event.SchemaAlteredEvent;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.util.IndexMetaDataUtil;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.handler.ddl.DropIndexStatementHandler;
@@ -49,40 +48,28 @@ public final class DropIndexStatementSchemaRefresher implements MetaDataRefreshe
     @Override
     public void refresh(final ShardingSphereMetaData schemaMetaData, final FederationDatabaseMetaData database, final Map<String, OptimizerPlannerContext> optimizerPlanners,
                         final Collection<String> logicDataSourceNames, final DropIndexStatement sqlStatement, final ConfigurationProperties props) throws SQLException {
-        Collection<String> indexNames = getIndexNames(sqlStatement);
-        Optional<SimpleTableSegment> simpleTableSegment = DropIndexStatementHandler.getSimpleTableSegment(sqlStatement);
-        String tableName = simpleTableSegment.map(tableSegment -> tableSegment.getTableName().getIdentifier().getValue()).orElse("");
-        TableMetaData tableMetaData = schemaMetaData.getDefaultSchema().get(tableName);
-        if (!Strings.isNullOrEmpty(tableName)) {
-            for (String each : indexNames) {
+        String logicTableName = getLogicTableName(schemaMetaData.getDefaultSchema(), sqlStatement).orElse("");
+        TableMetaData tableMetaData = schemaMetaData.getDefaultSchema().get(logicTableName);
+        if (null != tableMetaData) {
+            for (String each : getIndexNames(sqlStatement)) {
                 tableMetaData.getIndexes().remove(each);
             }
             post(schemaMetaData.getName(), tableMetaData);
-            return;
         }
-        for (String each : indexNames) {
-            Optional<String> logicTableNameOptional = findLogicTableName(schemaMetaData.getDefaultSchema(), each);
-            if (logicTableNameOptional.isPresent()) {
-                String logicTableName = logicTableNameOptional.orElse("");
-                Preconditions.checkArgument(!Strings.isNullOrEmpty(logicTableName), "Cannot get the table name!");
-                if (null == tableMetaData) {
-                    tableMetaData = schemaMetaData.getDefaultSchema().get(logicTableName);
-                }
-                Preconditions.checkNotNull(tableMetaData, "Cannot get the table metadata!");
-                tableMetaData.getIndexes().remove(each);
-            }
+    }
+    
+    private Optional<String> getLogicTableName(final ShardingSphereSchema schema, final DropIndexStatement sqlStatement) {
+        Optional<SimpleTableSegment> simpleTableSegment = DropIndexStatementHandler.getSimpleTableSegment(sqlStatement);
+        if (simpleTableSegment.isPresent()) {
+            return simpleTableSegment.map(optional -> optional.getTableName().getIdentifier().getValue());
         }
-        post(schemaMetaData.getName(), tableMetaData);
+        return IndexMetaDataUtil.getTableNamesFromMetaData(schema, sqlStatement.getIndexes()).stream().findFirst();
     }
     
     private Collection<String> getIndexNames(final DropIndexStatement dropIndexStatement) {
         return dropIndexStatement.getIndexes().stream().map(each -> each.getIdentifier().getValue()).collect(Collectors.toCollection(LinkedList::new));
     }
     
-    private Optional<String> findLogicTableName(final ShardingSphereSchema schema, final String logicIndexName) {
-        return schema.getAllTableNames().stream().filter(each -> schema.get(each).getIndexes().containsKey(logicIndexName)).findFirst();
-    }
-    
     private void post(final String schemaName, final TableMetaData tableMetaData) {
         SchemaAlteredEvent event = new SchemaAlteredEvent(schemaName);
         event.getAlteredTables().add(tableMetaData);
diff --git a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/route/SingleTableSQLRouter.java b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/route/SingleTableSQLRouter.java
index ded2140a3d4..55d6cd1955a 100644
--- a/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/route/SingleTableSQLRouter.java
+++ b/shardingsphere-kernel/shardingsphere-single-table/shardingsphere-single-table-core/src/main/java/org/apache/shardingsphere/singletable/route/SingleTableSQLRouter.java
@@ -21,22 +21,22 @@ import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.binder.type.IndexAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.metadata.schema.util.IndexMetaDataUtil;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.singletable.constant.SingleTableOrder;
 import org.apache.shardingsphere.singletable.rule.SingleTableRule;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateTableStatement;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 
 /**
  * Single table SQL router.
@@ -51,13 +51,14 @@ public final class SingleTableSQLRouter implements SQLRouter<SingleTableRule> {
             String actualDataSource = metaData.getResource().getDataSources().keySet().iterator().next();
             result.getRouteUnits().add(new RouteUnit(new RouteMapper(logicDataSource, actualDataSource), Collections.emptyList()));
         } else {
-            route(logicSQL.getSqlStatementContext(), rule, result, props);
+            route(logicSQL.getSqlStatementContext(), metaData.getDefaultSchema(), rule, props, result);
         }
         return result;
     }
     
-    private void route(final SQLStatementContext<?> sqlStatementContext, final SingleTableRule rule, final RouteContext routeContext, final ConfigurationProperties props) {
-        Collection<String> singleTableNames = getSingleTableNames(sqlStatementContext, rule, routeContext);
+    private void route(final SQLStatementContext<?> sqlStatementContext, final ShardingSphereSchema schema, final SingleTableRule rule, 
+                       final ConfigurationProperties props, final RouteContext routeContext) {
+        Collection<String> singleTableNames = getSingleTableNames(sqlStatementContext, schema, rule, routeContext);
         if (singleTableNames.isEmpty()) {
             return;
         }
@@ -65,17 +66,11 @@ public final class SingleTableSQLRouter implements SQLRouter<SingleTableRule> {
         new SingleTableRouteEngine(singleTableNames, sqlStatementContext.getSqlStatement()).route(routeContext, rule);
     }
     
-    private Collection<String> getSingleTableNames(final SQLStatementContext<?> sqlStatementContext, final SingleTableRule rule, final RouteContext routeContext) {
-        Collection<String> result;
-        if (sqlStatementContext instanceof TableAvailable) {
-            Collection<SimpleTableSegment> allTables = ((TableAvailable) sqlStatementContext).getAllTables();
-            result = new HashSet<>(allTables.size(), 1);
-            for (SimpleTableSegment each : allTables) {
-                String value = each.getTableName().getIdentifier().getValue();
-                result.add(value);
-            }
-        } else {
-            result = sqlStatementContext.getTablesContext().getTableNames();
+    private Collection<String> getSingleTableNames(final SQLStatementContext<?> sqlStatementContext, final ShardingSphereSchema schema, 
+                                                   final SingleTableRule rule, final RouteContext routeContext) {
+        Collection<String> result = sqlStatementContext.getTablesContext().getTableNames();
+        if (result.isEmpty() && sqlStatementContext instanceof IndexAvailable) {
+            result = IndexMetaDataUtil.getTableNamesFromMetaData(schema, ((IndexAvailable) sqlStatementContext).getIndexes());
         }
         return routeContext.getRouteUnits().isEmpty() && sqlStatementContext.getSqlStatement() instanceof CreateTableStatement ? result : rule.getSingleTableNames(result); 
     }
@@ -92,7 +87,7 @@ public final class SingleTableSQLRouter implements SQLRouter<SingleTableRule> {
     @Override
     public void decorateRouteContext(final RouteContext routeContext, final LogicSQL logicSQL, final ShardingSphereMetaData metaData,
                                      final SingleTableRule rule, final ConfigurationProperties props) {
-        route(logicSQL.getSqlStatementContext(), rule, routeContext, props);
+        route(logicSQL.getSqlStatementContext(), metaData.getDefaultSchema(), rule, props, routeContext);
     }
     
     @Override
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DDLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DDLStatement.g4
index 8d2d00e0912..f09503c9653 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DDLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DDLStatement.g4
@@ -149,7 +149,7 @@ alterTable
     ;
 
 alterIndex
-    : ALTER INDEX (existClause? | ALL IN TABLESPACE) indexName alterIndexDefinitionClause
+    : ALTER INDEX (existClause? | ALL IN TABLESPACE) qualifiedName alterIndexDefinitionClause
     ;
 
 dropTable
@@ -161,7 +161,7 @@ dropTableOpt
     ;
 
 dropIndex
-    : DROP INDEX concurrentlyClause existClause? indexNames dropIndexOpt?
+    : DROP INDEX concurrentlyClause existClause? qualifiedNameList dropIndexOpt?
     ;
 
 dropIndexOpt
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
index e38877606fb..bd781a42823 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
@@ -213,10 +213,6 @@ qualifiedNameList
     | qualifiedNameList COMMA_ qualifiedName
     ;
 
-qualifiedName
-    : colId | colId indirection
-    ;
-
 selectLimit
     : limitClause offsetClause
     | offsetClause limitClause
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
index 2bcc27d110e..85e21ee4f9b 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
@@ -470,22 +470,36 @@ public final class OpenGaussDDLStatementSQLVisitor extends OpenGaussStatementSQL
     @Override
     public ASTNode visitAlterIndex(final AlterIndexContext ctx) {
         OpenGaussAlterIndexStatement result = new OpenGaussAlterIndexStatement();
-        result.setIndex((IndexSegment) visit(ctx.indexName()));
+        result.setIndex(createIndexSegment((SimpleTableSegment) visit(ctx.qualifiedName())));
         if (null != ctx.alterIndexDefinitionClause().renameIndexSpecification()) {
             result.setRenameIndex((IndexSegment) visit(ctx.alterIndexDefinitionClause().renameIndexSpecification().indexName()));
         }
         return result;
     }
     
+    private IndexSegment createIndexSegment(final SimpleTableSegment tableSegment) {
+        IndexSegment result = new IndexSegment(tableSegment.getStartIndex(), tableSegment.getStopIndex(), tableSegment.getTableName().getIdentifier());
+        tableSegment.getOwner().ifPresent(result::setOwner);
+        return result;
+    }
+    
     @SuppressWarnings("unchecked")
     @Override
     public ASTNode visitDropIndex(final DropIndexContext ctx) {
         OpenGaussDropIndexStatement result = new OpenGaussDropIndexStatement();
-        result.getIndexes().addAll(((CollectionValue<IndexSegment>) visit(ctx.indexNames())).getValue());
+        result.getIndexes().addAll(createIndexSegments(((CollectionValue<SimpleTableSegment>) visit(ctx.qualifiedNameList())).getValue()));
         result.setContainsExistClause(null != ctx.existClause());
         return result;
     }
     
+    private Collection<IndexSegment> createIndexSegments(final Collection<SimpleTableSegment> tableSegments) {
+        Collection<IndexSegment> result = new LinkedList<>();
+        for (SimpleTableSegment each : tableSegments) {
+            result.add(createIndexSegment(each));
+        }
+        return result;
+    }
+    
     @Override
     public ASTNode visitIndexNames(final IndexNamesContext ctx) {
         CollectionValue<IndexSegment> result = new CollectionValue<>();
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
index de25051834d..d5abe182dd5 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
@@ -155,7 +155,7 @@ alterTable
     ;
 
 alterIndex
-    : ALTER INDEX (existClause? | ALL IN TABLESPACE) indexName alterIndexDefinitionClause
+    : ALTER INDEX (existClause? | ALL IN TABLESPACE) qualifiedName alterIndexDefinitionClause
     ;
 
 dropTable
@@ -167,7 +167,7 @@ dropTableOpt
     ;
 
 dropIndex
-    : DROP INDEX concurrentlyClause existClause? indexNames dropIndexOpt?
+    : DROP INDEX concurrentlyClause existClause? qualifiedNameList dropIndexOpt?
     ;
 
 dropIndexOpt
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
index 57389fce07e..e61d1216d8b 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
@@ -215,10 +215,6 @@ qualifiedNameList
     | qualifiedNameList COMMA_ qualifiedName
     ;
 
-qualifiedName
-    : colId | colId indirection
-    ;
-
 selectLimit
     : limitClause offsetClause
     | offsetClause limitClause
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
index 87baa476175..e0e8c3eb840 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
@@ -534,22 +534,36 @@ public final class PostgreSQLDDLStatementSQLVisitor extends PostgreSQLStatementS
     @Override
     public ASTNode visitAlterIndex(final AlterIndexContext ctx) {
         PostgreSQLAlterIndexStatement result = new PostgreSQLAlterIndexStatement();
-        result.setIndex((IndexSegment) visit(ctx.indexName()));
+        result.setIndex(createIndexSegment((SimpleTableSegment) visit(ctx.qualifiedName())));
         if (null != ctx.alterIndexDefinitionClause().renameIndexSpecification()) {
             result.setRenameIndex((IndexSegment) visit(ctx.alterIndexDefinitionClause().renameIndexSpecification().indexName()));
         }
         return result;
     }
     
+    private IndexSegment createIndexSegment(final SimpleTableSegment tableSegment) {
+        IndexSegment result = new IndexSegment(tableSegment.getStartIndex(), tableSegment.getStopIndex(), tableSegment.getTableName().getIdentifier());
+        tableSegment.getOwner().ifPresent(result::setOwner);
+        return result;
+    }
+    
     @SuppressWarnings("unchecked")
     @Override
     public ASTNode visitDropIndex(final DropIndexContext ctx) {
         PostgreSQLDropIndexStatement result = new PostgreSQLDropIndexStatement();
-        result.getIndexes().addAll(((CollectionValue<IndexSegment>) visit(ctx.indexNames())).getValue());
+        result.getIndexes().addAll(createIndexSegments(((CollectionValue<SimpleTableSegment>) visit(ctx.qualifiedNameList())).getValue()));
         result.setContainsExistClause(null != ctx.existClause());
         return result;
     }
     
+    private Collection<IndexSegment> createIndexSegments(final Collection<SimpleTableSegment> tableSegments) {
+        Collection<IndexSegment> result = new LinkedList<>();
+        for (SimpleTableSegment each : tableSegments) {
+            result.add(createIndexSegment(each));
+        }
+        return result;
+    }
+    
     @Override
     public ASTNode visitIndexNames(final IndexNamesContext ctx) {
         CollectionValue<IndexSegment> result = new CollectionValue<>();
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/alter-index.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/alter-index.xml
index 54d77afe550..a8908a2eab7 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/alter-index.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/alter-index.xml
@@ -79,5 +79,6 @@
     </alter-index>
     
     <alter-index sql-case-id="alter_index_alter_column" />
+    <alter-index sql-case-id="alter_index_with_schema" />
     <alter-index sql-case-id="alter_index_set_tablespace" />
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/drop-index.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/drop-index.xml
index cc59cdfb951..cba04974cb6 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/drop-index.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ddl/drop-index.xml
@@ -44,6 +44,7 @@
     
     <drop-index sql-case-id="drop_index_with_double_quota" />
     <drop-index sql-case-id="drop_index_concurrently" />
+    <drop-index sql-case-id="drop_index_with_schema" />
     
     <drop-index sql-case-id="drop_index_with_bracket">
         <table name="t_order" start-delimiter="[" end-delimiter="]" start-index="28" stop-index="36" />
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/alter-index.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/alter-index.xml
index 731d46aae5d..bcec55d5e7b 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/alter-index.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/alter-index.xml
@@ -36,4 +36,5 @@
     <sql-case id="alter_index_with_reorganize" value="ALTER INDEX order_index ON t_order REORGANIZE WITH (COMPRESS_ALL_ROW_GROUPS = ON)" db-types="SQLServer" />
     <sql-case id="alter_index_set_tablespace" value="ALTER INDEX distributors SET TABLESPACE fasttablespace" db-types="PostgreSQL,openGauss" />
     <sql-case id="alter_index_alter_column" value="ALTER INDEX t_order_idx ALTER COLUMN 3 SET STATISTICS 1000" db-types="PostgreSQL,openGauss"/>
+    <sql-case id="alter_index_with_schema" value="ALTER INDEX public.t_order_idx ALTER COLUMN 3 SET STATISTICS 1000" db-types="PostgreSQL,openGauss"/>
 </sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/drop-index.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/drop-index.xml
index c05f0b38078..65d46db449e 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/drop-index.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ddl/drop-index.xml
@@ -29,6 +29,7 @@
     <sql-case id="drop_index_with_quota" value="DROP INDEX &quot;order_index&quot; ON &quot;t_order&quot;" db-types="Oracle" />
     <sql-case id="drop_index_with_double_quota" value="DROP INDEX &quot;order_index&quot;" db-types="PostgreSQL,openGauss" />
     <sql-case id="drop_index_concurrently" value="DROP INDEX CONCURRENTLY order_index" db-types="PostgreSQL,openGauss" />
+    <sql-case id="drop_index_with_schema" value="DROP INDEX public.order_index" db-types="PostgreSQL,openGauss" />
     <sql-case id="drop_index_with_bracket" value="DROP INDEX [order_index] ON [t_order]" db-types="SQLServer" />
     <sql-case id="drop_index_if_exists_on_table" value="DROP INDEX IF EXISTS order_index ON t_order" db-types="SQLServer" />
     <sql-case id="drop_index_with_online_force_invalidation" value="DROP INDEX order_index ONLINE FORCE DEFERRED INVALIDATION" db-types="Oracle" />