You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dubbo.apache.org by li...@apache.org on 2018/11/01 07:20:11 UTC

[incubator-dubbo] 02/06: Change ZooKeeper's data structure as a config center

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

liujun pushed a commit to branch dev-metadata
in repository https://gitbox.apache.org/repos/asf/incubator-dubbo.git

commit d20a6aa5481b37392754429edfc0d150e0cc2b3e
Author: ken.lj <ke...@gmail.com>
AuthorDate: Thu Nov 1 10:47:11 2018 +0800

    Change ZooKeeper's data structure as a config center
---
 .../dubbo/config/context/ConfigurationWrapper.java | 53 ++++++++++++++++++++++
 .../apache/dubbo/config/context/Environment.java   | 28 ++++++++----
 .../support/apollo/ApolloDynamicConfiguration.java |  7 +++
 .../sources/ZooKeeperConfigurationSource.java      | 12 ++++-
 .../java/org/apache/dubbo/registry/ZKTools.java    | 21 ++++++++-
 5 files changed, 109 insertions(+), 12 deletions(-)

diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/ConfigurationWrapper.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/ConfigurationWrapper.java
new file mode 100644
index 0000000..2d758ae
--- /dev/null
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/ConfigurationWrapper.java
@@ -0,0 +1,53 @@
+/*
+ * 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.dubbo.config.context;
+
+import org.apache.dubbo.common.config.AbstractConfiguration;
+import org.apache.dubbo.common.config.Configuration;
+
+/**
+ *
+ */
+public class ConfigurationWrapper extends AbstractConfiguration {
+    private String application;
+    private String service;
+    private String method;
+
+    private Configuration delegate;
+
+    public ConfigurationWrapper(String application, String service, String method, Configuration configuration) {
+        this.application = application;
+        this.service = service;
+        this.method = method;
+        this.delegate = configuration;
+    }
+
+    @Override
+    protected Object getInternalProperty(String key) {
+        Object value = delegate.getProperty(application + "." + key);
+        if (value == null) {
+            value = delegate.getProperty(service + "." + key);
+        }
+        if (value == null) {
+            value = delegate.getProperty(service + "." + method + "." + key);
+        }
+        if (value == null) {
+            value = delegate.getProperty(key);
+        }
+        return value;
+    }
+}
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/Environment.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/Environment.java
index edd8134..db9daaa 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/Environment.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/context/Environment.java
@@ -79,17 +79,27 @@ public class Environment {
         });
     }
 
-    public CompositeConfiguration getRuntimeCompositeConf(URL url) {
-        return runtimeCompositeConfsHolder.computeIfAbsent(url.toIdentityString(), k -> {
-            CompositeConfiguration compositeConfiguration = new CompositeConfiguration();
-            compositeConfiguration.addConfiguration(getDynamicConfiguration());
-            compositeConfiguration.addConfiguration(this.getSystemConf(null, null));
-            compositeConfiguration.addConfiguration(url.toConfiguration());
-            compositeConfiguration.addConfiguration(this.getPropertiesConf(null, null));
-            return compositeConfiguration;
-        });
+    /**
+     * FIXME This method will recreate Configuration for each RPC, how much latency affect will this action has on performance?
+     *
+     * @param url
+     * @return
+     */
+    public CompositeConfiguration getRuntimeCompositeConf(URL url, String method) {
+        CompositeConfiguration compositeConfiguration = new CompositeConfiguration();
+
+        String app = url.getParameter(Constants.APPLICATION_KEY);
+        String service = url.getServiceKey();
+        compositeConfiguration.addConfiguration(new ConfigurationWrapper(app, service, method, getDynamicConfiguration()));
+
+        compositeConfiguration.addConfiguration(url.toConfiguration());
+        compositeConfiguration.addConfiguration(this.getSystemConf(null, null));
+        compositeConfiguration.addConfiguration(this.getPropertiesConf(null, null));
+        return compositeConfiguration;
     }
 
+
+
     /**
      * If user opens DynamicConfig, the extension instance must has been created during the initialization of ConfigCenterConfig with the right extension type user specified.
      * If no DynamicConfig presents, NopDynamicConfiguration will be used.
diff --git a/dubbo-governance/dubbo-governance-apollo/src/main/java/org/apache/dubbo/governance/support/apollo/ApolloDynamicConfiguration.java b/dubbo-governance/dubbo-governance-apollo/src/main/java/org/apache/dubbo/governance/support/apollo/ApolloDynamicConfiguration.java
index 14db27b..0c83088 100644
--- a/dubbo-governance/dubbo-governance-apollo/src/main/java/org/apache/dubbo/governance/support/apollo/ApolloDynamicConfiguration.java
+++ b/dubbo-governance/dubbo-governance-apollo/src/main/java/org/apache/dubbo/governance/support/apollo/ApolloDynamicConfiguration.java
@@ -98,6 +98,13 @@ public class ApolloDynamicConfiguration extends AbstractDynamicConfiguration<Con
      */
     @Override
     protected String getInternalProperty(String key, String group, long timeout) {
+        if (group != null) {
+            Config config = ConfigService.getConfig(group);
+            if (config != null) {
+                return config.getProperty(key, null);
+            }
+            return null;
+        }
         return dubboConfig.getProperty(key, null);
     }
 
diff --git a/dubbo-governance/dubbo-governance-zookeeper/src/main/java/org/apache/dubbo/governance/support/archaius/sources/ZooKeeperConfigurationSource.java b/dubbo-governance/dubbo-governance-zookeeper/src/main/java/org/apache/dubbo/governance/support/archaius/sources/ZooKeeperConfigurationSource.java
index 7a96c4d..7cdf108 100644
--- a/dubbo-governance/dubbo-governance-zookeeper/src/main/java/org/apache/dubbo/governance/support/archaius/sources/ZooKeeperConfigurationSource.java
+++ b/dubbo-governance/dubbo-governance-zookeeper/src/main/java/org/apache/dubbo/governance/support/archaius/sources/ZooKeeperConfigurationSource.java
@@ -55,6 +55,7 @@ public class ZooKeeperConfigurationSource implements WatchedConfigurationSource,
     private Executor executor = Executors.newFixedThreadPool(1);
     private final CuratorFramework client;
 
+    // The final root path would be: /configRootPath/"config"
     private final String configRootPath;
     private final TreeCache treeCache;
     private boolean connected = false;
@@ -72,13 +73,19 @@ public class ZooKeeperConfigurationSource implements WatchedConfigurationSource,
     }
 
 
+    /**
+     * @param connectString,  the zookeeper address
+     * @param sessionTimeout, timeout for session
+     * @param connectTimeout, timeout to wait before build a connection
+     * @param configRootPath, the final path would be: configRootPath/"config"
+     */
     public ZooKeeperConfigurationSource(String connectString, int sessionTimeout, int connectTimeout, String configRootPath) {
         if (connectString == null) {
             throw new IllegalArgumentException("connectString==null, must specify the address to connect for zookeeper archaius source.");
         }
 
         if (!configRootPath.startsWith("/")) {
-            configRootPath = "/" + configRootPath;
+            configRootPath = "/" + configRootPath + "/config";
         }
 
         CuratorFramework client = CuratorFrameworkFactory.newClient(connectString, sessionTimeout, connectTimeout,
@@ -136,6 +143,7 @@ public class ZooKeeperConfigurationSource implements WatchedConfigurationSource,
                     return;
                 }
 
+                // TODO?
                 if (data.getPath().split("/").length == 5) {
                     byte[] value = data.getData();
                     String stringValue = new String(value, charset);
@@ -191,7 +199,7 @@ public class ZooKeeperConfigurationSource implements WatchedConfigurationSource,
         Map<String, Object> all = new HashMap<>();
 
         if (!connected) {
-            logger.warn("ConfigCenter is not connected yet, zookeeper don't support local snapshot yet, so there's no old data to use!");
+            logger.warn("ConfigCenter is not connected yet, zookeeper does't support local snapshot, so there's no backup data to use!");
             return all;
         }
 
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
index e310349..e113b93 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
@@ -43,13 +43,32 @@ public class ZKTools {
                 new ExponentialBackoffRetry(1000, 3));
         client.start();
 
-        testProviderConfig();
+        testStartupConfig();
+//        testProviderConfig();
 //        testPathCache();
 //        testTreeCache();
 //        testCuratorListener();
         System.in.read();
     }
 
+    public static void testStartupConfig() {
+        String str = "dubbo.registry.address=\"zookeeper://127.0.0.1:2181\"\n" +
+                "dubbo.registry.group=\"dubboregistrygroup1\"\n" +
+                "dubbo.servicestore.address=\"zookeeper://127.0.0.1:2181\"\n" +
+                "dubbo.protocol.port=20990\n" +
+                "dubbo.service.org.apache.dubbo.demo.DemoService.timeout=9999\n";
+
+        try {
+            String path = "/dubboregistrygroup1/dubbo.properties";
+            if (client.checkExists().forPath(path) == null) {
+                client.create().creatingParentsIfNeeded().forPath(path);
+            }
+            setData(path, str);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
     public static void testProviderConfig() {
         String str = "{\n" +
                 "\t\"service\": \"org.apache.dubbo.demo.DemoService\",\n" +