You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2023/02/06 12:26:23 UTC

[shardingsphere] branch master updated: Add TableNameReviser (#24030)

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

duanzhengqiang 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 ce6d108dd07 Add TableNameReviser (#24030)
ce6d108dd07 is described below

commit ce6d108dd071b3ee82ce159881995495948cf94a
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Mon Feb 6 20:26:12 2023 +0800

    Add TableNameReviser (#24030)
---
 .../metadata/EncryptSchemaMetaDataDecorator.java   | 11 +---
 .../metadata/ShardingSchemaMetaDataDecorator.java  | 18 +++---
 .../metadata/reviser/ShardingTableNameReviser.java | 36 ++++++++++++
 .../decorator/reviser/table/TableNameReviser.java  | 32 ++++++++++
 .../decorator/reviser/table/TableReviseEngine.java | 68 ++++++++++++++++++++++
 .../metadata/SingleSchemaMetaDataDecorator.java    | 12 ++--
 6 files changed, 150 insertions(+), 27 deletions(-)

diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptSchemaMetaDataDecorator.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptSchemaMetaDataDecorator.java
index 6b4a38bc158..d60736d0522 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptSchemaMetaDataDecorator.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/metadata/EncryptSchemaMetaDataDecorator.java
@@ -22,8 +22,8 @@ import org.apache.shardingsphere.encrypt.metadata.reviser.EncryptColumnNameRevis
 import org.apache.shardingsphere.encrypt.rule.EncryptRule;
 import org.apache.shardingsphere.encrypt.rule.EncryptTable;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviser;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.table.TableReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.spi.RuleBasedSchemaMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
@@ -34,7 +34,6 @@ import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 
 /**
  * Schema meta data decorator for encrypt.
@@ -55,12 +54,8 @@ public final class EncryptSchemaMetaDataDecorator implements RuleBasedSchemaMeta
     }
     
     private TableMetaData decorate(final String tableName, final TableMetaData tableMetaData, final EncryptRule encryptRule) {
-        Optional<EncryptTable> encryptTable = encryptRule.findEncryptTable(tableName);
-        if (!encryptTable.isPresent()) {
-            return tableMetaData;
-        }
-        Collection<ColumnReviser> revisers = getColumnRevisers(encryptTable.get());
-        return new TableMetaData(tableName, new ColumnReviseEngine().revise(tableMetaData.getColumns(), revisers), tableMetaData.getIndexes(), tableMetaData.getConstrains());
+        return encryptRule.findEncryptTable(tableName)
+                .map(optional -> new TableReviseEngine().revise(tableMetaData, getColumnRevisers(optional), Collections.emptyList(), Collections.emptyList())).orElse(tableMetaData);
     }
     
     private Collection<ColumnReviser> getColumnRevisers(final EncryptTable encryptTable) {
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingSchemaMetaDataDecorator.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingSchemaMetaDataDecorator.java
index b7737346943..5ce1a2aa1de 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingSchemaMetaDataDecorator.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/ShardingSchemaMetaDataDecorator.java
@@ -19,17 +19,17 @@ package org.apache.shardingsphere.sharding.metadata;
 
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviseEngine;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.constraint.ConstraintReviseEngine;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.index.IndexReviseEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.table.TableReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.spi.RuleBasedSchemaMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.exception.metadata.InconsistentShardingTableMetaDataException;
 import org.apache.shardingsphere.sharding.metadata.reviser.ShardingColumnGeneratedReviser;
 import org.apache.shardingsphere.sharding.metadata.reviser.ShardingConstraintReviser;
 import org.apache.shardingsphere.sharding.metadata.reviser.ShardingIndexReviser;
+import org.apache.shardingsphere.sharding.metadata.reviser.ShardingTableNameReviser;
 import org.apache.shardingsphere.sharding.rule.ShardingRule;
 import org.apache.shardingsphere.sharding.rule.TableRule;
 
@@ -68,17 +68,15 @@ public final class ShardingSchemaMetaDataDecorator implements RuleBasedSchemaMet
     }
     
     private TableMetaData createTableMetaData(final ShardingRule rule, final TableRule tableRule, final TableMetaData tableMetaData) {
-        return new TableMetaData(tableRule.getLogicTable(), new ColumnReviseEngine().revise(tableMetaData.getColumns(), Collections.singleton(new ShardingColumnGeneratedReviser(tableRule))),
-                new IndexReviseEngine().revise(tableMetaData.getName(), tableMetaData.getIndexes(), Collections.singleton(new ShardingIndexReviser(tableRule))),
-                new ConstraintReviseEngine().revise(tableMetaData.getName(), tableMetaData.getConstrains(), Collections.singleton(new ShardingConstraintReviser(rule, tableRule))));
+        return new TableReviseEngine().revise(tableMetaData, new ShardingTableNameReviser(tableRule), Collections.singleton(new ShardingColumnGeneratedReviser(tableRule)),
+                Collections.singleton(new ShardingIndexReviser(tableRule)), Collections.singleton(new ShardingConstraintReviser(rule, tableRule)));
     }
     
     private Map<String, Collection<TableMetaData>> getLogicTableMetaDataMap(final SchemaMetaData schemaMetaData, final ShardingRule rule) {
         Map<String, Collection<TableMetaData>> result = new LinkedHashMap<>();
         for (TableMetaData each : schemaMetaData.getTables()) {
             String logicTableName = rule.findLogicTableByActualTable(each.getName()).orElse(each.getName());
-            Collection<TableMetaData> tableMetaDataList = result.computeIfAbsent(logicTableName, key -> new LinkedList<>());
-            tableMetaDataList.add(decorate(each, rule));
+            result.computeIfAbsent(logicTableName, key -> new LinkedList<>()).add(decorate(each, rule));
         }
         return result;
     }
@@ -87,9 +85,7 @@ public final class ShardingSchemaMetaDataDecorator implements RuleBasedSchemaMet
         TableMetaData sample = tableMetaDataList.iterator().next();
         Collection<TableMetaDataViolation> violations = tableMetaDataList.stream()
                 .filter(each -> !sample.equals(each)).map(each -> new TableMetaDataViolation(each.getName(), each)).collect(Collectors.toList());
-        if (!violations.isEmpty()) {
-            throw new InconsistentShardingTableMetaDataException(logicTableName, violations);
-        }
+        ShardingSpherePreconditions.checkState(violations.isEmpty(), () -> new InconsistentShardingTableMetaDataException(logicTableName, violations));
     }
     
     @Override
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/ShardingTableNameReviser.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/ShardingTableNameReviser.java
new file mode 100644
index 00000000000..1024c7851fc
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/reviser/ShardingTableNameReviser.java
@@ -0,0 +1,36 @@
+/*
+ * 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.shardingsphere.sharding.metadata.reviser;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.table.TableNameReviser;
+import org.apache.shardingsphere.sharding.rule.TableRule;
+
+/**
+ * Sharding table name reviser.
+ */
+@RequiredArgsConstructor
+public final class ShardingTableNameReviser implements TableNameReviser {
+    
+    private final TableRule tableRule;
+    
+    @Override
+    public String revise(final String originalName) {
+        return tableRule.getLogicTable();
+    }
+}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableNameReviser.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableNameReviser.java
new file mode 100644
index 00000000000..818d4d3e0c5
--- /dev/null
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableNameReviser.java
@@ -0,0 +1,32 @@
+/*
+ * 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.shardingsphere.infra.metadata.database.schema.decorator.reviser.table;
+
+/**
+ * Table name reviser.
+ */
+public interface TableNameReviser {
+    
+    /**
+     * Revise table meta data.
+     * 
+     * @param originalName original table name
+     * @return revised table name
+     */
+    String revise(String originalName);
+}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableReviseEngine.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableReviseEngine.java
new file mode 100644
index 00000000000..973e58cd2e8
--- /dev/null
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/decorator/reviser/table/TableReviseEngine.java
@@ -0,0 +1,68 @@
+/*
+ * 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.shardingsphere.infra.metadata.database.schema.decorator.reviser.table;
+
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviseEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviser;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.constraint.ConstraintReviseEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.constraint.ConstraintReviser;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.index.IndexReviseEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.index.IndexReviser;
+import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
+
+import java.util.Collection;
+
+/**
+ * Table revise engine.
+ */
+public final class TableReviseEngine {
+    
+    /**
+     * Revise table meta data.
+     *
+     * @param originalMetaData original table meta data
+     * @param columnRevisers column revisers
+     * @param indexRevisers index revisers
+     * @param constraintRevisers constraint revisers
+     * @return revised table meta data
+     */
+    public TableMetaData revise(final TableMetaData originalMetaData,
+                                final Collection<ColumnReviser> columnRevisers, final Collection<IndexReviser> indexRevisers, final Collection<ConstraintReviser> constraintRevisers) {
+        return new TableMetaData(originalMetaData.getName(), new ColumnReviseEngine().revise(originalMetaData.getColumns(), columnRevisers),
+                new IndexReviseEngine().revise(originalMetaData.getName(), originalMetaData.getIndexes(), indexRevisers),
+                new ConstraintReviseEngine().revise(originalMetaData.getName(), originalMetaData.getConstrains(), constraintRevisers));
+    }
+    
+    /**
+     * Revise table meta data.
+     * 
+     * @param originalMetaData original table meta data
+     * @param tableNameReviser table name reviser
+     * @param columnRevisers column revisers
+     * @param indexRevisers index revisers
+     * @param constraintRevisers constraint revisers
+     * @return revised table meta data
+     */
+    public TableMetaData revise(final TableMetaData originalMetaData, final TableNameReviser tableNameReviser,
+                                final Collection<ColumnReviser> columnRevisers, final Collection<IndexReviser> indexRevisers, final Collection<ConstraintReviser> constraintRevisers) {
+        String revisedTableName = tableNameReviser.revise(originalMetaData.getName());
+        return new TableMetaData(revisedTableName, new ColumnReviseEngine().revise(originalMetaData.getColumns(), columnRevisers),
+                new IndexReviseEngine().revise(revisedTableName, originalMetaData.getIndexes(), indexRevisers),
+                new ConstraintReviseEngine().revise(revisedTableName, originalMetaData.getConstrains(), constraintRevisers));
+    }
+}
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/SingleSchemaMetaDataDecorator.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/SingleSchemaMetaDataDecorator.java
index 11fe8632ab0..97ee0b79308 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/SingleSchemaMetaDataDecorator.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/SingleSchemaMetaDataDecorator.java
@@ -18,9 +18,7 @@
 package org.apache.shardingsphere.single.metadata;
 
 import org.apache.shardingsphere.infra.metadata.database.schema.builder.GenericSchemaBuilderMaterial;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.column.ColumnReviseEngine;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.constraint.ConstraintReviseEngine;
-import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.index.IndexReviseEngine;
+import org.apache.shardingsphere.infra.metadata.database.schema.decorator.reviser.table.TableReviseEngine;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.spi.RuleBasedSchemaMetaDataDecorator;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.SchemaMetaData;
 import org.apache.shardingsphere.infra.metadata.database.schema.loader.model.TableMetaData;
@@ -47,17 +45,15 @@ public final class SingleSchemaMetaDataDecorator implements RuleBasedSchemaMetaD
         for (Entry<String, SchemaMetaData> entry : schemaMetaDataMap.entrySet()) {
             Collection<TableMetaData> tables = new LinkedList<>();
             for (TableMetaData each : entry.getValue().getTables()) {
-                tables.add(decorate(each.getName(), each));
+                tables.add(decorate(each));
             }
             result.put(entry.getKey(), new SchemaMetaData(entry.getKey(), tables));
         }
         return result;
     }
     
-    private TableMetaData decorate(final String tableName, final TableMetaData tableMetaData) {
-        return new TableMetaData(tableName, new ColumnReviseEngine().revise(tableMetaData.getColumns(), Collections.emptyList()),
-                new IndexReviseEngine().revise(tableMetaData.getName(), tableMetaData.getIndexes(), Collections.singleton(new SingleIndexReviser())),
-                new ConstraintReviseEngine().revise(tableMetaData.getName(), tableMetaData.getConstrains(), Collections.singleton(new SingleConstraintReviser())));
+    private TableMetaData decorate(final TableMetaData tableMetaData) {
+        return new TableReviseEngine().revise(tableMetaData, Collections.emptyList(), Collections.singleton(new SingleIndexReviser()), Collections.singleton(new SingleConstraintReviser()));
     }
     
     @Override