You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2023/06/20 05:28:39 UTC

[shardingsphere] branch master updated: Refactor SingleRule to new metadata structure (#26442)

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

zhaojinchao 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 090d188e0ec Refactor SingleRule to new metadata structure (#26442)
090d188e0ec is described below

commit 090d188e0ec4899221a6e3581c2535e2245d2505
Author: ChenJiaHao <Pa...@163.com>
AuthorDate: Tue Jun 20 13:28:33 2023 +0800

    Refactor SingleRule to new metadata structure (#26442)
---
 .../metadata/converter/BroadcastNodeConverter.java |   4 +-
 kernel/single/core/pom.xml                         |   5 +
 .../event/SingleRuleConfigurationEventBuilder.java |  41 +++++----
 .../single/event/config/AddSingleTableEvent.java   |  35 +++++++
 .../single/event/config/AlterSingleTableEvent.java |  35 +++++++
 .../event/config/DeleteSingleTableEvent.java       |  32 +++++++
 .../metadata/converter/SingleNodeConverter.java    |  18 ++--
 .../subscriber/SingleConfigurationSubscriber.java  | 102 +++++++++++++++++++++
 .../NewYamlSingleRuleConfigurationSwapper.java     |   5 +-
 ...nfra.rule.RuleConfigurationSubscribeCoordinator |  18 ++++
 10 files changed, 262 insertions(+), 33 deletions(-)

diff --git a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java
index 9378c59f6c2..78efb88b514 100644
--- a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java
+++ b/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java
@@ -38,9 +38,9 @@ public final class BroadcastNodeConverter {
     private static final String VERSION_PATTERN = "/versions/[0-9]+";
     
     /**
-     * Get table name path.
+     * Get tables path.
      *
-     * @return table name path
+     * @return tables path
      */
     public static String getTablesPath() {
         return TABLES_NODE;
diff --git a/kernel/single/core/pom.xml b/kernel/single/core/pom.xml
index e8b08a37d2a..e7af1fed236 100644
--- a/kernel/single/core/pom.xml
+++ b/kernel/single/core/pom.xml
@@ -49,6 +49,11 @@
             <artifactId>shardingsphere-infra-context</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-mode-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/SingleRuleConfigurationEventBuilder.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/SingleRuleConfigurationEventBuilder.java
index 26113ded3cf..b0a746176e4 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/SingleRuleConfigurationEventBuilder.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/SingleRuleConfigurationEventBuilder.java
@@ -18,18 +18,17 @@
 package org.apache.shardingsphere.single.event;
 
 import com.google.common.base.Strings;
-import org.apache.shardingsphere.infra.config.rule.global.converter.GlobalRuleNodeConverter;
-import org.apache.shardingsphere.infra.config.rule.global.event.AlterGlobalRuleConfigurationEvent;
-import org.apache.shardingsphere.infra.config.rule.global.event.DeleteGlobalRuleConfigurationEvent;
 import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.mode.event.DataChangedEvent;
 import org.apache.shardingsphere.mode.event.DataChangedEvent.Type;
 import org.apache.shardingsphere.mode.spi.RuleConfigurationEventBuilder;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
-import org.apache.shardingsphere.single.rule.SingleRule;
+import org.apache.shardingsphere.single.event.config.AddSingleTableEvent;
+import org.apache.shardingsphere.single.event.config.AlterSingleTableEvent;
+import org.apache.shardingsphere.single.event.config.DeleteSingleTableEvent;
+import org.apache.shardingsphere.single.metadata.converter.SingleNodeConverter;
 import org.apache.shardingsphere.single.yaml.config.pojo.YamlSingleRuleConfiguration;
-import org.apache.shardingsphere.single.yaml.config.swapper.YamlSingleRuleConfigurationSwapper;
 
 import java.util.Optional;
 
@@ -38,30 +37,34 @@ import java.util.Optional;
  */
 public final class SingleRuleConfigurationEventBuilder implements RuleConfigurationEventBuilder {
     
-    private static final String SINGLE = "single";
-    
-    private static final String RULE_TYPE = SingleRule.class.getSimpleName();
-    
     @Override
     public Optional<GovernanceEvent> build(final String databaseName, final DataChangedEvent event) {
-        if (!GlobalRuleNodeConverter.isExpectedRuleName(SINGLE, event.getKey()) || Strings.isNullOrEmpty(event.getValue())) {
+        if (!SingleNodeConverter.isSinglePath(event.getKey()) || Strings.isNullOrEmpty(event.getValue())) {
             return Optional.empty();
         }
-        Optional<String> version = GlobalRuleNodeConverter.getVersion(SINGLE, event.getKey());
-        if (version.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
-            return buildEvent(event, Integer.parseInt(version.get()));
+        if (SingleNodeConverter.isTablesPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createSingleConfigEvent(databaseName, event);
         }
         return Optional.empty();
     }
     
-    private Optional<GovernanceEvent> buildEvent(final DataChangedEvent event, final int version) {
-        if (Type.ADDED == event.getType() || Type.UPDATED == event.getType()) {
-            return Optional.of(new AlterGlobalRuleConfigurationEvent(swapToConfig(event.getValue()), RULE_TYPE, event.getKey(), version));
+    private Optional<GovernanceEvent> createSingleConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddSingleTableEvent(databaseName, swapSingleTableRuleConfig(event.getValue())));
         }
-        return Optional.of(new DeleteGlobalRuleConfigurationEvent(RULE_TYPE, event.getKey(), version));
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterSingleTableEvent(databaseName, swapSingleTableRuleConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteSingleTableEvent(databaseName));
     }
     
-    private SingleRuleConfiguration swapToConfig(final String yamlContext) {
-        return new YamlSingleRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlSingleRuleConfiguration.class));
+    private SingleRuleConfiguration swapSingleTableRuleConfig(final String yamlContext) {
+        SingleRuleConfiguration result = new SingleRuleConfiguration();
+        YamlSingleRuleConfiguration yamlSingleRuleConfiguration = YamlEngine.unmarshal(yamlContext, YamlSingleRuleConfiguration.class);
+        if (null != yamlSingleRuleConfiguration.getTables()) {
+            result.getTables().addAll(yamlSingleRuleConfiguration.getTables());
+        }
+        result.setDefaultDataSource(yamlSingleRuleConfiguration.getDefaultDataSource());
+        return result;
     }
 }
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AddSingleTableEvent.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AddSingleTableEvent.java
new file mode 100644
index 00000000000..86591f2c0e8
--- /dev/null
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AddSingleTableEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.single.event.config;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
+
+/**
+ * Add single table event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddSingleTableEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final SingleRuleConfiguration config;
+}
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AlterSingleTableEvent.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AlterSingleTableEvent.java
new file mode 100644
index 00000000000..19919b8decd
--- /dev/null
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/AlterSingleTableEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.single.event.config;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
+
+/**
+ * Alter single table event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterSingleTableEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final SingleRuleConfiguration config;
+}
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/DeleteSingleTableEvent.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/DeleteSingleTableEvent.java
new file mode 100644
index 00000000000..ab0a404a680
--- /dev/null
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/event/config/DeleteSingleTableEvent.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.single.event.config;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete single table event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteSingleTableEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/converter/SingleNodeConverter.java
similarity index 84%
copy from features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java
copy to kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/converter/SingleNodeConverter.java
index 9378c59f6c2..322fd30c851 100644
--- a/features/broadcast/core/src/main/java/org/apache/shardingsphere/broadcast/metadata/converter/BroadcastNodeConverter.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/metadata/converter/SingleNodeConverter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.broadcast.metadata.converter;
+package org.apache.shardingsphere.single.metadata.converter;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
@@ -24,12 +24,12 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
- * Broadcast node converter.
+ * Single node converter.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class BroadcastNodeConverter {
+public final class SingleNodeConverter {
     
-    private static final String ROOT_NODE = "broadcast";
+    private static final String ROOT_NODE = "single";
     
     private static final String TABLES_NODE = "tables";
     
@@ -38,28 +38,28 @@ public final class BroadcastNodeConverter {
     private static final String VERSION_PATTERN = "/versions/[0-9]+";
     
     /**
-     * Get table name path.
+     * Get tables path.
      *
-     * @return table name path
+     * @return tables path
      */
     public static String getTablesPath() {
         return TABLES_NODE;
     }
     
     /**
-     * Is broadcast path.
+     * Is single path.
      *
      * @param rulePath rule path
      * @return true or false
      */
-    public static boolean isBroadcastPath(final String rulePath) {
+    public static boolean isSinglePath(final String rulePath) {
         Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
         Matcher matcher = pattern.matcher(rulePath);
         return matcher.find();
     }
     
     /**
-     * Is broadcast tables path.
+     * Is tables path.
      *
      * @param rulePath rule path
      * @return true or false
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/subscriber/SingleConfigurationSubscriber.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/subscriber/SingleConfigurationSubscriber.java
new file mode 100644
index 00000000000..928f279282d
--- /dev/null
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/subscriber/SingleConfigurationSubscriber.java
@@ -0,0 +1,102 @@
+/*
+ * 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.single.subscriber;
+
+import com.google.common.eventbus.Subscribe;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator;
+import org.apache.shardingsphere.mode.event.config.DatabaseRuleConfigurationChangedEvent;
+import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
+import org.apache.shardingsphere.single.event.config.AddSingleTableEvent;
+import org.apache.shardingsphere.single.event.config.AlterSingleTableEvent;
+import org.apache.shardingsphere.single.event.config.DeleteSingleTableEvent;
+import org.apache.shardingsphere.single.rule.SingleRule;
+
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Single configuration subscriber.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@RequiredArgsConstructor
+public final class SingleConfigurationSubscriber implements RuleConfigurationSubscribeCoordinator {
+    
+    private Map<String, ShardingSphereDatabase> databases;
+    
+    private InstanceContext instanceContext;
+    
+    @Override
+    public void registerRuleConfigurationSubscriber(final Map<String, ShardingSphereDatabase> databases, final InstanceContext instanceContext) {
+        this.databases = databases;
+        this.instanceContext = instanceContext;
+        instanceContext.getEventBusContext().register(this);
+    }
+    
+    /**
+     * Renew with add single configuration.
+     *
+     * @param event add single configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddSingleTableEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        SingleRuleConfiguration needToAddedConfig = event.getConfig();
+        Optional<SingleRule> rule = database.getRuleMetaData().findSingleRule(SingleRule.class);
+        SingleRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = rule.get().getConfiguration();
+            config.getTables().clear();
+            config.getTables().addAll(needToAddedConfig.getTables());
+        } else {
+            config = new SingleRuleConfiguration(needToAddedConfig.getTables(), needToAddedConfig.getDefaultDataSource().orElse(null));
+        }
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter single configuration.
+     *
+     * @param event alter single configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterSingleTableEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        SingleRuleConfiguration needToAlteredConfig = event.getConfig();
+        SingleRuleConfiguration config = database.getRuleMetaData().getSingleRule(SingleRule.class).getConfiguration();
+        config.setTables(needToAlteredConfig.getTables());
+        config.setDefaultDataSource(needToAlteredConfig.getDefaultDataSource().orElse(null));
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete single configuration.
+     *
+     * @param event delete single configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteSingleTableEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        SingleRuleConfiguration config = database.getRuleMetaData().getSingleRule(SingleRule.class).getConfiguration();
+        config.getTables().clear();
+        config.setDefaultDataSource(null);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+}
diff --git a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/yaml/config/swapper/NewYamlSingleRuleConfigurationSwapper.java b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/yaml/config/swapper/NewYamlSingleRuleConfigurationSwapper.java
index dd49edf9b84..67cf184d7cb 100644
--- a/kernel/single/core/src/main/java/org/apache/shardingsphere/single/yaml/config/swapper/NewYamlSingleRuleConfigurationSwapper.java
+++ b/kernel/single/core/src/main/java/org/apache/shardingsphere/single/yaml/config/swapper/NewYamlSingleRuleConfigurationSwapper.java
@@ -23,6 +23,7 @@ import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
 import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
 import org.apache.shardingsphere.single.api.config.SingleRuleConfiguration;
 import org.apache.shardingsphere.single.constant.SingleOrder;
+import org.apache.shardingsphere.single.metadata.converter.SingleNodeConverter;
 import org.apache.shardingsphere.single.yaml.config.pojo.YamlSingleRuleConfiguration;
 
 import java.util.Collection;
@@ -34,12 +35,10 @@ import java.util.Collections;
  */
 public final class NewYamlSingleRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<SingleRuleConfiguration> {
     
-    private static final String ROOT_NODE = "tables";
-    
     @Override
     public Collection<YamlDataNode> swapToDataNodes(final SingleRuleConfiguration data) {
         // TODO Consider whether to split tables and defaultDataSource
-        return Collections.singletonList(new YamlDataNode(ROOT_NODE, YamlEngine.marshal(swapToYamlConfiguration(data))));
+        return Collections.singletonList(new YamlDataNode(SingleNodeConverter.getTablesPath(), YamlEngine.marshal(swapToYamlConfiguration(data))));
     }
     
     private YamlSingleRuleConfiguration swapToYamlConfiguration(final SingleRuleConfiguration data) {
diff --git a/kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator b/kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator
new file mode 100644
index 00000000000..5280c5c5221
--- /dev/null
+++ b/kernel/single/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.single.subscriber.SingleConfigurationSubscriber