You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2021/04/29 15:23:17 UTC

[shardingsphere] branch master updated: Add GovernanceListenerFactory for SPI to introduce GovernanceListener (#10219)

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

wuweijie 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 19d65ff  Add GovernanceListenerFactory for SPI to introduce GovernanceListener (#10219)
19d65ff is described below

commit 19d65ff896bb31a686a6b8c0061c91d7ef57b57a
Author: Liang Zhang <te...@163.com>
AuthorDate: Thu Apr 29 23:22:39 2021 +0800

    Add GovernanceListenerFactory for SPI to introduce GovernanceListener (#10219)
    
    * Add GovernanceListenerFactory
    
    * Add GovernanceListenerFactory impl via SPI
---
 .../event/listener/GovernanceListenerFactory.java  | 45 ++++++++++++++++++
 .../registry/listener/RegistryListenerManager.java | 36 ++++++---------
 .../DataSourceStateChangedListenerFactory.java     | 43 ++++++++++++++++++
 .../factory/LockChangedListenerFactory.java        | 43 ++++++++++++++++++
 .../PrivilegeNodeChangedListenerFactory.java       | 43 ++++++++++++++++++
 .../factory/PropertiesChangedListenerFactory.java  | 43 ++++++++++++++++++
 .../TerminalStateChangedListenerFactory.java       | 43 ++++++++++++++++++
 .../factory/UserChangedListenerFactory.java        | 43 ++++++++++++++++++
 ...e.core.event.listener.GovernanceListenerFactory | 23 ++++++++++
 .../listener/RegistryListenerManagerTest.java      | 53 ----------------------
 10 files changed, 341 insertions(+), 74 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/listener/GovernanceListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/listener/GovernanceListenerFactory.java
new file mode 100644
index 0000000..5831e46
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/listener/GovernanceListenerFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.governance.core.event.listener;
+
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Collection;
+
+/**
+ * Governance listener factory.
+ */
+public interface GovernanceListenerFactory {
+    
+    /**
+     * Create governance listener.
+     * 
+     * @param registryRepository registry repository
+     * @param schemaNames schema names
+     * @return governance listener
+     */
+    GovernanceListener create(RegistryRepository registryRepository, Collection<String> schemaNames);
+    
+    /**
+     * Get watch types.
+     * 
+     * @return watch types
+     */
+    Collection<Type> getWatchTypes();
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManager.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManager.java
index 1455e2f..c8d97f9 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManager.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManager.java
@@ -17,9 +17,11 @@
 
 package org.apache.shardingsphere.governance.core.registry.listener;
 
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
 import org.apache.shardingsphere.governance.core.registry.listener.metadata.MetaDataListener;
 import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
 import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 
 import java.util.Collection;
 
@@ -28,40 +30,32 @@ import java.util.Collection;
  */
 public final class RegistryListenerManager {
     
-    private final TerminalStateChangedListener terminalStateChangedListener;
+    static {
+        ShardingSphereServiceLoader.register(GovernanceListenerFactory.class);
+    }
     
-    private final DataSourceStateChangedListener dataSourceStateChangedListener;
+    private final RegistryRepository registryRepository;
+    
+    private final Collection<String> schemaNames;
     
-    private final LockChangedListener lockChangedListener;
-
     private final MetaDataListener metaDataListener;
-
-    private final PropertiesChangedListener propertiesChangedListener;
-
-    private final UserChangedListener userChangedListener;
     
-    private final PrivilegeNodeChangedListener privilegeNodeChangedListener;
+    private final Collection<GovernanceListenerFactory> governanceListenerFactories;
     
     public RegistryListenerManager(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
-        terminalStateChangedListener = new TerminalStateChangedListener(registryRepository);
-        dataSourceStateChangedListener = new DataSourceStateChangedListener(registryRepository, schemaNames);
-        lockChangedListener = new LockChangedListener(registryRepository);
+        this.registryRepository = registryRepository;
+        this.schemaNames = schemaNames;
         metaDataListener = new MetaDataListener(registryRepository, schemaNames);
-        propertiesChangedListener = new PropertiesChangedListener(registryRepository);
-        userChangedListener = new UserChangedListener(registryRepository);
-        privilegeNodeChangedListener = new PrivilegeNodeChangedListener(registryRepository);
+        governanceListenerFactories = ShardingSphereServiceLoader.getSingletonServiceInstances(GovernanceListenerFactory.class);
     }
     
     /**
      * Initialize all state changed listeners.
      */
     public void initListeners() {
-        terminalStateChangedListener.watch(Type.UPDATED);
-        dataSourceStateChangedListener.watch(Type.UPDATED, Type.DELETED, Type.ADDED);
-        lockChangedListener.watch(Type.ADDED, Type.DELETED);
         metaDataListener.watch();
-        propertiesChangedListener.watch(Type.UPDATED);
-        userChangedListener.watch(Type.UPDATED);
-        privilegeNodeChangedListener.watch(Type.UPDATED);
+        for (GovernanceListenerFactory each : governanceListenerFactories) {
+            each.create(registryRepository, schemaNames).watch(each.getWatchTypes().toArray(new Type[0]));
+        }
     }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/DataSourceStateChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/DataSourceStateChangedListenerFactory.java
new file mode 100644
index 0000000..d82a2bd
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/DataSourceStateChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.DataSourceStateChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Data source state changed listener factory.
+ */
+public final class DataSourceStateChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new DataSourceStateChangedListener(registryRepository, schemaNames);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Arrays.asList(Type.ADDED, Type.UPDATED, Type.DELETED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/LockChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/LockChangedListenerFactory.java
new file mode 100644
index 0000000..15aeb3e
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/LockChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.LockChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Lock changed listener factory.
+ */
+public final class LockChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new LockChangedListener(registryRepository);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Arrays.asList(Type.ADDED, Type.DELETED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PrivilegeNodeChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PrivilegeNodeChangedListenerFactory.java
new file mode 100644
index 0000000..78fa790
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PrivilegeNodeChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.PrivilegeNodeChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Privilege changed listener factory.
+ */
+public final class PrivilegeNodeChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new PrivilegeNodeChangedListener(registryRepository);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Collections.singleton(Type.UPDATED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PropertiesChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PropertiesChangedListenerFactory.java
new file mode 100644
index 0000000..7ff99e4
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/PropertiesChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.PropertiesChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Properties changed listener factory.
+ */
+public final class PropertiesChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new PropertiesChangedListener(registryRepository);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Collections.singleton(Type.UPDATED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/TerminalStateChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/TerminalStateChangedListenerFactory.java
new file mode 100644
index 0000000..6246926
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/TerminalStateChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.TerminalStateChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Terminal state changed listener factory.
+ */
+public final class TerminalStateChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new TerminalStateChangedListener(registryRepository);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Collections.singleton(Type.UPDATED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/UserChangedListenerFactory.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/UserChangedListenerFactory.java
new file mode 100644
index 0000000..3709e17
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/registry/listener/factory/UserChangedListenerFactory.java
@@ -0,0 +1,43 @@
+/*
+ * 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.governance.core.registry.listener.factory;
+
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListener;
+import org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory;
+import org.apache.shardingsphere.governance.core.registry.listener.UserChangedListener;
+import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * User changed listener factory.
+ */
+public final class UserChangedListenerFactory implements GovernanceListenerFactory {
+    
+    @Override
+    public GovernanceListener create(final RegistryRepository registryRepository, final Collection<String> schemaNames) {
+        return new UserChangedListener(registryRepository);
+    }
+    
+    @Override
+    public Collection<Type> getWatchTypes() {
+        return Collections.singleton(Type.UPDATED);
+    }
+}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory b/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory
new file mode 100644
index 0000000..88153df
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.governance.core.event.listener.GovernanceListenerFactory
@@ -0,0 +1,23 @@
+#
+# 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.governance.core.registry.listener.factory.TerminalStateChangedListenerFactory
+org.apache.shardingsphere.governance.core.registry.listener.factory.DataSourceStateChangedListenerFactory
+org.apache.shardingsphere.governance.core.registry.listener.factory.LockChangedListenerFactory
+org.apache.shardingsphere.governance.core.registry.listener.factory.PropertiesChangedListenerFactory
+org.apache.shardingsphere.governance.core.registry.listener.factory.UserChangedListenerFactory
+org.apache.shardingsphere.governance.core.registry.listener.factory.PrivilegeNodeChangedListenerFactory
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManagerTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManagerTest.java
deleted file mode 100644
index c5fb69b..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/registry/listener/RegistryListenerManagerTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.governance.core.registry.listener;
-
-import org.apache.shardingsphere.governance.core.registry.util.FieldUtil;
-import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
-import org.apache.shardingsphere.governance.repository.api.listener.DataChangedEvent.Type;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.junit.MockitoJUnitRunner;
-
-import java.util.Arrays;
-
-import static org.mockito.Mockito.verify;
-
-@RunWith(MockitoJUnitRunner.class)
-public final class RegistryListenerManagerTest {
-    
-    @Mock
-    private RegistryRepository registryRepository;
-    
-    @Mock
-    private TerminalStateChangedListener terminalStateChangedListener;
-    
-    @Mock
-    private DataSourceStateChangedListener dataSourceStateChangedListener;
-    
-    @Test
-    public void assertInitListeners() {
-        RegistryListenerManager actual = new RegistryListenerManager(registryRepository, Arrays.asList("sharding_db", "replica_query_db", "encrypt_db"));
-        FieldUtil.setField(actual, "terminalStateChangedListener", terminalStateChangedListener);
-        FieldUtil.setField(actual, "dataSourceStateChangedListener", dataSourceStateChangedListener);
-        actual.initListeners();
-        verify(terminalStateChangedListener).watch(Type.UPDATED);
-        verify(dataSourceStateChangedListener).watch(Type.UPDATED, Type.DELETED, Type.ADDED);
-    }
-}