You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2020/02/04 12:04:39 UTC

[GitHub] [incubator-shardingsphere] dongzl opened a new pull request #4166: refactor orchestration config for apollo.

dongzl opened a new pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166
 
 
   For #3185 .
   
   Changes proposed in this pull request:
   - fixes orchestration config listener for nacos and apollo.
   - refactor orchestration config for apollo.
   - fixes nacos config key name.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375041793
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
+        namespace = config.getNamespace();
+        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
+        env = properties.getProperty("env", "DEV");
+        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
+        administrator = properties.getProperty("administrator");
+        String apolloToken = properties.getProperty("token");
+        String portalUrl = properties.getProperty("portalUrl");
+        Integer connectTimeout = Ints.tryParse(Objects.toString(properties.get("connectTimeout")));
+        Integer readTimeout = Ints.tryParse(Objects.toString(properties.get("readTimeout")));
+        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
+                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
+                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
+                .withToken(apolloToken).build();
+    }
+    
+    /**
+     * Get config value by key.
+     * 
+     * @param key key
+     * @return value
+     */
+    public String getValue(final String key) {
+        OpenItemDTO itemDTO = client.getItem(appId, env, clusterName, namespace, key);
+        if (itemDTO == null) {
+            return null;
+        }
+        return itemDTO.getValue();
+    }
+    
+    /**
+     * Persist config.
+     * 
+     * @param key key
+     * @param value value
+     */
+    public void persist(final String key, final String value) {
 
 Review comment:
   I think we can implement the function and optimize it in the future. For Apollo, if publishNameSpace fail, user can manual management in the Apollo's portal.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] coveralls edited a comment on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-584065767
 
 
   ## Pull Request Test Coverage Report for [Build 9427](https://coveralls.io/builds/28629085)
   
   * **45** of **97**   **(46.39%)**  changed or added relevant lines in **8** files are covered.
   * **2** unchanged lines in **1** file lost coverage.
   * Overall coverage decreased (**-0.08%**) to **65.027%**
   
   ---
   
   |  Changes Missing Coverage | Covered Lines | Changed/Added Lines | % |
   | :-----|--------------|--------|---: |
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-api/src/main/java/org/apache/shardingsphere/orchestration/center/util/ConfigKeyUtils.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-api%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Futil%2FConfigKeyUtils.java#L25) | 3 | 4 | 75.0%
   | [sharding-orchestration/sharding-orchestration-core/src/main/java/org/apache/shardingsphere/orchestration/internal/registry/config/node/ConfigurationNode.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Finternal%2Fregistry%2Fconfig%2Fnode%2FConfigurationNode.java#L143) | 5 | 6 | 83.33%
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2FApolloInstance.java#L60) | 13 | 18 | 72.22%
   | [sharding-orchestration/sharding-orchestration-core/src/main/java/org/apache/shardingsphere/orchestration/internal/registry/config/listener/SchemaChangedListener.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Finternal%2Fregistry%2Fconfig%2Flistener%2FSchemaChangedListener.java#L81) | 1 | 10 | 10.0%
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2Fwrapper%2FApolloOpenApiWrapper.java#L50) | 0 | 36 | 0.0%
   <!-- | **Total:** | **45** | **97** | **46.39%** | -->
   
   |  Files with Coverage Reduction | New Missed Lines | % |
   | :-----|--------------|--: |
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java](https://coveralls.io/builds/28629085/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2FApolloInstance.java#L62) | 2 | 65.63% |
   <!-- | **Total:** | **2** |  | -->
   
   |  Totals | [![Coverage Status](https://coveralls.io/builds/28629085/badge)](https://coveralls.io/builds/28629085) |
   | :-- | --: |
   | Change from base [Build 9423](https://coveralls.io/builds/28627782): |  -0.08% |
   | Covered Lines: | 11013 |
   | Relevant Lines: | 16936 |
   
   ---
   ##### 💛  - [Coveralls](https://coveralls.io)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-585058548
 
 
   Close this pr, and submit a new pr to master branch directly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375038901
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/test/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapperTest.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.enums.PropertyChangeType;
+import com.ctrip.framework.apollo.mockserver.EmbeddedApollo;
+import com.ctrip.framework.apollo.model.ConfigChangeEvent;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ApolloConfigWrapperTest {
+    
+    static {
+        SLF4JBridgeHandler.removeHandlersForRootLogger();
 
 Review comment:
   Apollo use java.util.log, it is in conflict with shardingsphere, so it should be remove.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375013287
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapper.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.Config;
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.ConfigService;
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Apollo config wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloConfigWrapper {
+    
+    private static final String APOLLO_KEY_APP_ID = "app.id";
+    
+    private static final String APOLLO_KEY_ENV = "env";
+    
+    private Config apolloConfig;
+    
+    public ApolloConfigWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   Does this function need a document comment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl closed pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl closed pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r374714353
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 
 Review comment:
   There are not log print, so I remove this Annotation.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375042428
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
 
 Review comment:
   I don't think there's any difference between them, I think thery are OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r374715461
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/test/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapperTest.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.enums.PropertyChangeType;
+import com.ctrip.framework.apollo.mockserver.EmbeddedApollo;
+import com.ctrip.framework.apollo.model.ConfigChangeEvent;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ApolloConfigWrapperTest {
+    
+    static {
+        SLF4JBridgeHandler.removeHandlersForRootLogger();
 
 Review comment:
   For close java.util.log print, only use logback.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-583190548
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375011671
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
+        return Strings.isNullOrEmpty(value) ? openApiWrapper.getValue(ConfigKeyUtils.path2Key(key)) : value;
     }
     
     @Override
     public List<String> getChildrenKeys(final String key) {
 
 Review comment:
   I think maybe we could remove getChildrenKeys from config center interface.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375012393
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
+        return Strings.isNullOrEmpty(value) ? openApiWrapper.getValue(ConfigKeyUtils.path2Key(key)) : value;
     }
     
     @Override
     public List<String> getChildrenKeys(final String key) {
-        return new ArrayList<>(tree.getChildrenKeys(key));
+        return null;
     }
     
     @Override
     public void persist(final String key, final String value) {
-        String apolloKey = convertKey(key);
-        updateKey(apolloKey, value);
-        publishNamespace();
-        tree.refresh(apolloKey, APOLLO_KEY_SEPARATOR);
-    }
-    
-    private void updateKey(final String key, final String value) {
-        OpenItemDTO openItem = new OpenItemDTO();
-        openItem.setKey(key);
-        openItem.setValue(value);
-        openItem.setComment("ShardingSphere create or update config");
-        openItem.setDataChangeCreatedBy(administrator);
-        client.createOrUpdateItem(appId, env, clusterName, namespace, openItem);
-    }
-    
-    private void publishNamespace() {
-        NamespaceReleaseDTO release = new NamespaceReleaseDTO();
-        release.setReleaseTitle("ShardingSphere namespace release");
-        release.setReleaseComment("ShardingSphere namespace release");
-        release.setReleasedBy(administrator);
-        release.setEmergencyPublish(true);
-        client.publishNamespace(appId, env, clusterName, namespace, release);
+        openApiWrapper.persist(ConfigKeyUtils.path2Key(key), value);
     }
     
     @Override
     public void watch(final String key, final DataChangedEventListener dataChangedEventListener) {
-        apolloConfig.addChangeListener(new ConfigChangeListener() {
+        String apolloKey = ConfigKeyUtils.path2Key(key);
+        caches.put(apolloKey, dataChangedEventListener);
+        ConfigChangeListener listener = new ConfigChangeListener() {
+
             @Override
             public void onChange(final ConfigChangeEvent changeEvent) {
-                for (String key : changeEvent.changedKeys()) {
-                    ConfigChange change = changeEvent.getChange(key);
+                for (String changeKey : changeEvent.changedKeys()) {
+                    ConfigChange change = changeEvent.getChange(changeKey);
                     DataChangedEvent.ChangedType changedType = getChangedType(change.getChangeType());
-                    if (DataChangedEvent.ChangedType.IGNORED != changedType) {
-                        dataChangedEventListener.onChange(new DataChangedEvent(deConvertKey(key), change.getNewValue(), changedType));
+                    if (DataChangedEvent.ChangedType.IGNORED == changedType) {
+                        continue;
+                    }
+                    if (caches.get(changeKey) == null) {
+                        continue;
                     }
+                    caches.get(changeKey).onChange(new DataChangedEvent(ConfigKeyUtils.key2Path(changeKey), change.getNewValue(), changedType));
 
 Review comment:
   Is this line too long? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375040252
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-api/src/main/java/org/apache/shardingsphere/orchestration/center/util/ConfigKeyUtils.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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.orchestration.center.util;
+
+/**
+ * Config key utils.
+ *
+ * @author dongzonglei
+ */
+public final class ConfigKeyUtils {
 
 Review comment:
   This is only use for orchestration ConfigCenter module right now, I think we will remove it to a common module in the future If need.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375015460
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
+        return Strings.isNullOrEmpty(value) ? openApiWrapper.getValue(ConfigKeyUtils.path2Key(key)) : value;
     }
     
     @Override
     public List<String> getChildrenKeys(final String key) {
 
 Review comment:
   And close() method also could be discussed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r374672385
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/test/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapperTest.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.enums.PropertyChangeType;
+import com.ctrip.framework.apollo.mockserver.EmbeddedApollo;
+import com.ctrip.framework.apollo.model.ConfigChangeEvent;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ApolloConfigWrapperTest {
+    
+    static {
+        SLF4JBridgeHandler.removeHandlersForRootLogger();
 
 Review comment:
   why remove?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375043486
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
+        return Strings.isNullOrEmpty(value) ? openApiWrapper.getValue(ConfigKeyUtils.path2Key(key)) : value;
     }
     
     @Override
     public List<String> getChildrenKeys(final String key) {
 
 Review comment:
   The etcd does not integrate now, I think if it doesn't need getChildrenKeys, we can remove it.
   
   close() method should be retain, but it is not need for all framework.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375017348
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-nacos/pom.xml
 ##########
 @@ -39,7 +35,7 @@
         <dependency>
             <groupId>com.alibaba.nacos</groupId>
             <artifactId>nacos-client</artifactId>
-            <version>${nacos-client.verison}</version>
+            <version>${nacos.client.verison}</version>
             <scope>provided</scope>
 
 Review comment:
   Why is provided?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375040348
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   The same to above.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl removed a comment on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl removed a comment on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-583190548
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375013484
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   Does this function need a document comment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375032432
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-api/src/main/java/org/apache/shardingsphere/orchestration/center/util/ConfigKeyUtils.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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.orchestration.center.util;
+
+/**
+ * Config key utils.
+ *
+ * @author dongzonglei
+ */
+public final class ConfigKeyUtils {
 
 Review comment:
   Whether should provide public utility method in interface module?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375016328
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/test/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapperTest.java
 ##########
 @@ -0,0 +1,101 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.enums.PropertyChangeType;
+import com.ctrip.framework.apollo.mockserver.EmbeddedApollo;
+import com.ctrip.framework.apollo.model.ConfigChangeEvent;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ApolloConfigWrapperTest {
+    
+    static {
+        SLF4JBridgeHandler.removeHandlersForRootLogger();
 
 Review comment:
   Dose apollo default use the logback to print log?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375010993
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
 
 Review comment:
   String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
   or
   String apolloKey = ConfigKeyUtils.path2Key(key);
   String value = configWrapper.getProperty(newKey);
   Which one is better, accroding to clean code principle?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375039628
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   The same to above.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r374669085
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 
 Review comment:
   why remove slf4j?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375015001
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
+        namespace = config.getNamespace();
+        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
+        env = properties.getProperty("env", "DEV");
+        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
+        administrator = properties.getProperty("administrator");
+        String apolloToken = properties.getProperty("token");
+        String portalUrl = properties.getProperty("portalUrl");
+        Integer connectTimeout = Ints.tryParse(Objects.toString(properties.get("connectTimeout")));
+        Integer readTimeout = Ints.tryParse(Objects.toString(properties.get("readTimeout")));
+        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
+                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
+                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
+                .withToken(apolloToken).build();
+    }
+    
+    /**
+     * Get config value by key.
+     * 
+     * @param key key
+     * @return value
+     */
+    public String getValue(final String key) {
+        OpenItemDTO itemDTO = client.getItem(appId, env, clusterName, namespace, key);
+        if (itemDTO == null) {
+            return null;
+        }
+        return itemDTO.getValue();
+    }
+    
+    /**
+     * Persist config.
+     * 
+     * @param key key
+     * @param value value
+     */
+    public void persist(final String key, final String value) {
 
 Review comment:
   Does this need some code to make sure that updateKey could rollback when publishNameSpace fail? For example: TCC.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375040700
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-nacos/pom.xml
 ##########
 @@ -39,7 +35,7 @@
         <dependency>
             <groupId>com.alibaba.nacos</groupId>
             <artifactId>nacos-client</artifactId>
-            <version>${nacos-client.verison}</version>
+            <version>${nacos.client.verison}</version>
             <scope>provided</scope>
 
 Review comment:
   nacos is not a default implemention, it provides for user selection, so it is provided.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] codecov-io commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-584080717
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`orchestration-5.x@e03143c`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `60.7%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/graphs/tree.svg?width=650&token=ZvlXpWa7so&height=150&src=pr)](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=tree)
   
   ```diff
   @@                 Coverage Diff                 @@
   ##             orchestration-5.x   #4166   +/-   ##
   ===================================================
     Coverage                     ?   61.1%           
     Complexity                   ?     352           
   ===================================================
     Files                        ?    1013           
     Lines                        ?   16936           
     Branches                     ?    2981           
   ===================================================
     Hits                         ?   10348           
     Misses                       ?    5933           
     Partials                     ?     655
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...core/strategy/route/none/NoneShardingStrategy.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS9zdHJhdGVneS9yb3V0ZS9ub25lL05vbmVTaGFyZGluZ1N0cmF0ZWd5LmphdmE=) | `100% <ø> (ø)` | `0 <0> (?)` | |
   | [.../masterslave/LoadBalanceStrategyConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWFwaS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvYXBpL2NvbmZpZy9tYXN0ZXJzbGF2ZS9Mb2FkQmFsYW5jZVN0cmF0ZWd5Q29uZmlndXJhdGlvbi5qYXZh) | `50% <ø> (ø)` | `1 <0> (?)` | |
   | [...route/time/exception/TimeServiceInitException.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9kYXRhYmFzZS10aW1lLXNlcnZpY2Uvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL3JvdXRlL3RpbWUvZXhjZXB0aW9uL1RpbWVTZXJ2aWNlSW5pdEV4Y2VwdGlvbi5qYXZh) | `0% <ø> (ø)` | `0 <0> (?)` | |
   | [...config/common/YamlAuthenticationConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9jb21tb24vWWFtbEF1dGhlbnRpY2F0aW9uQ29uZmlndXJhdGlvbi5qYXZh) | `100% <ø> (ø)` | `0 <0> (?)` | |
   | [...ig/sharding/YamlShardingStrategyConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9zaGFyZGluZy9ZYW1sU2hhcmRpbmdTdHJhdGVneUNvbmZpZ3VyYXRpb24uamF2YQ==) | `100% <ø> (ø)` | `1 <0> (?)` | |
   | [...core/strategy/route/hint/HintShardingStrategy.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS9zdHJhdGVneS9yb3V0ZS9oaW50L0hpbnRTaGFyZGluZ1N0cmF0ZWd5LmphdmE=) | `0% <ø> (ø)` | `0 <0> (?)` | |
   | [...config/sharding/YamlKeyGeneratorConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9zaGFyZGluZy9ZYW1sS2V5R2VuZXJhdG9yQ29uZmlndXJhdGlvbi5qYXZh) | `100% <ø> (ø)` | `0 <0> (?)` | |
   | [...config/sharding/YamlShardingRuleConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9zaGFyZGluZy9ZYW1sU2hhcmRpbmdSdWxlQ29uZmlndXJhdGlvbi5qYXZh) | `100% <ø> (ø)` | `0 <0> (?)` | |
   | [.../yaml/config/common/YamlRootRuleConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9jb21tb24vWWFtbFJvb3RSdWxlQ29uZmlndXJhdGlvbi5qYXZh) | `100% <ø> (ø)` | `0 <0> (?)` | |
   | [...ml/config/sharding/YamlTableRuleConfiguration.java](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree#diff-c2hhcmRpbmctY29yZS9zaGFyZGluZy1jb3JlLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2hhcmRpbmdzcGhlcmUvY29yZS95YW1sL2NvbmZpZy9zaGFyZGluZy9ZYW1sVGFibGVSdWxlQ29uZmlndXJhdGlvbi5qYXZh) | `100% <ø> (ø)` | `1 <0> (?)` | |
   | ... and [121 more](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=footer). Last update [e03143c...51eab05](https://codecov.io/gh/apache/incubator-shardingsphere/pull/4166?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
kimmking commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r377027007
 
 

 ##########
 File path: sharding-proxy/sharding-proxy-bootstrap/src/main/resources/conf/server.yaml
 ##########
 @@ -29,11 +29,6 @@
 #    namespace: orchestration
 #    props:
 #      overwrite: false
-#      retry-interval-milliseconds: 10
 
 Review comment:
   why remove?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-582274029
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375010993
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
 
 Review comment:
   String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
   or
   String newKey = ConfigKeyUtils.path2Key(key);
   String value = configWrapper.getProperty(newKey);
   Which one is better, accroding to clean code principle?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
wgy8283335 commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375010993
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
 
 Review comment:
   String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
   or
   String newKey = ConfigKeyUtils.path2Key(key);
   String value = configWrapper.getProperty(newKey);
   Which one is better, accroding to clean code?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-582283703
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] coveralls commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
coveralls commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-584065767
 
 
   ## Pull Request Test Coverage Report for [Build 1671](https://coveralls.io/builds/28628264)
   
   * **45** of **97**   **(46.39%)**  changed or added relevant lines in **8** files are covered.
   * **4** unchanged lines in **2** files lost coverage.
   * Overall coverage decreased (**-0.06%**) to **65.039%**
   
   ---
   
   |  Changes Missing Coverage | Covered Lines | Changed/Added Lines | % |
   | :-----|--------------|--------|---: |
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-api/src/main/java/org/apache/shardingsphere/orchestration/center/util/ConfigKeyUtils.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-api%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Futil%2FConfigKeyUtils.java#L25) | 3 | 4 | 75.0%
   | [sharding-orchestration/sharding-orchestration-core/src/main/java/org/apache/shardingsphere/orchestration/internal/registry/config/node/ConfigurationNode.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Finternal%2Fregistry%2Fconfig%2Fnode%2FConfigurationNode.java#L143) | 5 | 6 | 83.33%
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2FApolloInstance.java#L60) | 13 | 18 | 72.22%
   | [sharding-orchestration/sharding-orchestration-core/src/main/java/org/apache/shardingsphere/orchestration/internal/registry/config/listener/SchemaChangedListener.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Finternal%2Fregistry%2Fconfig%2Flistener%2FSchemaChangedListener.java#L81) | 1 | 10 | 10.0%
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2Fwrapper%2FApolloOpenApiWrapper.java#L50) | 0 | 36 | 0.0%
   <!-- | **Total:** | **45** | **97** | **46.39%** | -->
   
   |  Files with Coverage Reduction | New Missed Lines | % |
   | :-----|--------------|--: |
   | [sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-center%2Fsharding-orchestration-center-apollo%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Fcenter%2Finstance%2FApolloInstance.java#L62) | 2 | 68.75% |
   | [sharding-orchestration/sharding-orchestration-core/src/main/java/org/apache/shardingsphere/orchestration/internal/util/IpUtils.java](https://coveralls.io/builds/28628264/source?filename=sharding-orchestration%2Fsharding-orchestration-core%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fshardingsphere%2Forchestration%2Finternal%2Futil%2FIpUtils.java#L74) | 2 | 80.0% |
   <!-- | **Total:** | **4** |  | -->
   
   |  Totals | [![Coverage Status](https://coveralls.io/builds/28628264/badge)](https://coveralls.io/builds/28628264) |
   | :-- | --: |
   | Change from base [Build 9423](https://coveralls.io/builds/28627782): |  -0.06% |
   | Covered Lines: | 11015 |
   | Relevant Lines: | 16936 |
   
   ---
   ##### 💛  - [Coveralls](https://coveralls.io)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375039559
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloConfigWrapper.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.Config;
+import com.ctrip.framework.apollo.ConfigChangeListener;
+import com.ctrip.framework.apollo.ConfigService;
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Apollo config wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloConfigWrapper {
+    
+    private static final String APOLLO_KEY_APP_ID = "app.id";
+    
+    private static final String APOLLO_KEY_ENV = "env";
+    
+    private Config apolloConfig;
+    
+    public ApolloConfigWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   This is a construct method,  construct method doesn't need comment on ShardingSphere.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375043757
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/ApolloInstance.java
 ##########
 @@ -17,177 +17,93 @@
 
 package org.apache.shardingsphere.orchestration.center.instance;
 
-import com.ctrip.framework.apollo.Config;
 import com.ctrip.framework.apollo.ConfigChangeListener;
-import com.ctrip.framework.apollo.ConfigService;
-import com.ctrip.framework.apollo.core.ConfigConsts;
 import com.ctrip.framework.apollo.enums.PropertyChangeType;
 import com.ctrip.framework.apollo.model.ConfigChange;
 import com.ctrip.framework.apollo.model.ConfigChangeEvent;
-import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
-import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
-import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
-import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Ints;
 import lombok.Getter;
 import lombok.Setter;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.orchestration.center.api.ConfigCenter;
 import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
-import org.apache.shardingsphere.orchestration.center.instance.node.ConfigTreeNode;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloConfigWrapper;
+import org.apache.shardingsphere.orchestration.center.instance.wrapper.ApolloOpenApiWrapper;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEvent;
 import org.apache.shardingsphere.orchestration.center.listener.DataChangedEventListener;
+import org.apache.shardingsphere.orchestration.center.util.ConfigKeyUtils;
 
-import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
 
 /**
- * Registry center for Apollo.
+ * Config center for Apollo.
  *
  * @author dongzonglei
  */
-@Slf4j
 public final class ApolloInstance implements ConfigCenter {
     
-    private static final String SHARDING_SPHERE_KEY_ROOT = "/";
+    private final Map<String, DataChangedEventListener> caches = new HashMap<>();
     
-    private static final String SHARDING_SPHERE_KEY_SEPARATOR = "/";
+    private ApolloConfigWrapper configWrapper;
     
-    private static final String APOLLO_KEY_SEPARATOR = ".";
-    
-    private static final String APOLLO_KEY_APP_ID = "app.id";
-    
-    private static final String APOLLO_KEY_ENV = "env";
-    
-    private static final String APOLLO_KEY_CLUSTER = ConfigConsts.APOLLO_CLUSTER_KEY;
-    
-    private static final String APOLLO_KEY_META = ConfigConsts.APOLLO_META_KEY;
-    
-    private String namespace;
-    
-    private String appId;
-    
-    private String env;
-    
-    private String clusterName;
-    
-    private String administrator;
-    
-    private Config apolloConfig;
-    
-    private ApolloOpenApiClient client;
-    
-    private ConfigTreeNode tree;
+    private ApolloOpenApiWrapper openApiWrapper;
     
     @Getter
     @Setter
     private Properties properties = new Properties();
     
     @Override
     public void init(final InstanceConfiguration config) {
-        initApolloConfig(config);
-        initApolloOpenApiClient();
-        initKeysRelationship();
-    }
-    
-    private void initApolloConfig(final InstanceConfiguration config) {
-        namespace = config.getNamespace();
-        appId = properties.getProperty("appId", "APOLLO_SHARDING_SPHERE");
-        env = properties.getProperty("env", "DEV");
-        clusterName = properties.getProperty("clusterName", ConfigConsts.CLUSTER_NAME_DEFAULT);
-        System.setProperty(APOLLO_KEY_APP_ID, appId);
-        System.setProperty(APOLLO_KEY_ENV, env);
-        System.setProperty(APOLLO_KEY_CLUSTER, clusterName);
-        System.setProperty(APOLLO_KEY_META, config.getServerLists());
-        apolloConfig = ConfigService.getConfig(namespace);
-    }
-    
-    private void initApolloOpenApiClient() {
-        administrator = properties.getProperty("administrator");
-        String apolloToken = properties.getProperty("token");
-        String portalUrl = properties.getProperty("portalUrl");
-        Integer connectTimeout = Ints.tryParse(properties.getProperty("connectTimeout"));
-        Integer readTimeout = Ints.tryParse(properties.getProperty("readTimeout"));
-        client = ApolloOpenApiClient.newBuilder().withPortalUrl(portalUrl)
-                .withConnectTimeout(connectTimeout == null ? ApolloOpenApiConstants.DEFAULT_CONNECT_TIMEOUT : connectTimeout)
-                .withReadTimeout(readTimeout == null ? ApolloOpenApiConstants.DEFAULT_READ_TIMEOUT : readTimeout)
-                .withToken(apolloToken).build();
-    }
-    
-    private void initKeysRelationship() {
-        List<OpenItemDTO> items = client.getNamespace(appId, env, clusterName, namespace).getItems();
-        Set<String> keys = Sets.newHashSet();
-        for (OpenItemDTO each : items) {
-            keys.add(each.getKey());
-        }
-        tree = ConfigTreeNode.create(keys, ".");
+        configWrapper = new ApolloConfigWrapper(config, properties);
+        openApiWrapper = new ApolloOpenApiWrapper(config, properties);
     }
     
     @Override
     public String get(final String key) {
-        return apolloConfig.getProperty(convertKey(key), "");
-    }
-    
-    private String convertKey(final String shardingSphereKey) {
-        return shardingSphereKey.replace(SHARDING_SPHERE_KEY_SEPARATOR, APOLLO_KEY_SEPARATOR).substring(1);
-    }
-    
-    private String deConvertKey(final String apolloKey) {
-        return new StringBuilder(SHARDING_SPHERE_KEY_ROOT).append(apolloKey.replace(APOLLO_KEY_SEPARATOR, SHARDING_SPHERE_KEY_SEPARATOR)).toString();
+        String value = configWrapper.getProperty(ConfigKeyUtils.path2Key(key));
+        return Strings.isNullOrEmpty(value) ? openApiWrapper.getValue(ConfigKeyUtils.path2Key(key)) : value;
     }
     
     @Override
     public List<String> getChildrenKeys(final String key) {
-        return new ArrayList<>(tree.getChildrenKeys(key));
+        return null;
     }
     
     @Override
     public void persist(final String key, final String value) {
-        String apolloKey = convertKey(key);
-        updateKey(apolloKey, value);
-        publishNamespace();
-        tree.refresh(apolloKey, APOLLO_KEY_SEPARATOR);
-    }
-    
-    private void updateKey(final String key, final String value) {
-        OpenItemDTO openItem = new OpenItemDTO();
-        openItem.setKey(key);
-        openItem.setValue(value);
-        openItem.setComment("ShardingSphere create or update config");
-        openItem.setDataChangeCreatedBy(administrator);
-        client.createOrUpdateItem(appId, env, clusterName, namespace, openItem);
-    }
-    
-    private void publishNamespace() {
-        NamespaceReleaseDTO release = new NamespaceReleaseDTO();
-        release.setReleaseTitle("ShardingSphere namespace release");
-        release.setReleaseComment("ShardingSphere namespace release");
-        release.setReleasedBy(administrator);
-        release.setEmergencyPublish(true);
-        client.publishNamespace(appId, env, clusterName, namespace, release);
+        openApiWrapper.persist(ConfigKeyUtils.path2Key(key), value);
     }
     
     @Override
     public void watch(final String key, final DataChangedEventListener dataChangedEventListener) {
-        apolloConfig.addChangeListener(new ConfigChangeListener() {
+        String apolloKey = ConfigKeyUtils.path2Key(key);
+        caches.put(apolloKey, dataChangedEventListener);
+        ConfigChangeListener listener = new ConfigChangeListener() {
+
             @Override
             public void onChange(final ConfigChangeEvent changeEvent) {
-                for (String key : changeEvent.changedKeys()) {
-                    ConfigChange change = changeEvent.getChange(key);
+                for (String changeKey : changeEvent.changedKeys()) {
+                    ConfigChange change = changeEvent.getChange(changeKey);
                     DataChangedEvent.ChangedType changedType = getChangedType(change.getChangeType());
-                    if (DataChangedEvent.ChangedType.IGNORED != changedType) {
-                        dataChangedEventListener.onChange(new DataChangedEvent(deConvertKey(key), change.getNewValue(), changedType));
+                    if (DataChangedEvent.ChangedType.IGNORED == changedType) {
+                        continue;
+                    }
+                    if (caches.get(changeKey) == null) {
+                        continue;
                     }
+                    caches.get(changeKey).onChange(new DataChangedEvent(ConfigKeyUtils.key2Path(changeKey), change.getNewValue(), changedType));
 
 Review comment:
   ShardingSphere's checkstyle does not warn, so it is in one line.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] kimmking commented on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
kimmking commented on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-585054391
 
 
   @dongzl  you can close this pr and submit it to master directly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl removed a comment on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl removed a comment on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-582274029
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl removed a comment on issue #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl removed a comment on issue #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#issuecomment-582283703
 
 
   /run ci

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-shardingsphere] dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.

Posted by GitBox <gi...@apache.org>.
dongzl commented on a change in pull request #4166: refactor orchestration config for apollo.
URL: https://github.com/apache/incubator-shardingsphere/pull/4166#discussion_r375040348
 
 

 ##########
 File path: sharding-orchestration/sharding-orchestration-center/sharding-orchestration-center-apollo/src/main/java/org/apache/shardingsphere/orchestration/center/instance/wrapper/ApolloOpenApiWrapper.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.orchestration.center.instance.wrapper;
+
+import com.ctrip.framework.apollo.core.ConfigConsts;
+import com.ctrip.framework.apollo.openapi.client.ApolloOpenApiClient;
+import com.ctrip.framework.apollo.openapi.client.constant.ApolloOpenApiConstants;
+import com.ctrip.framework.apollo.openapi.dto.NamespaceReleaseDTO;
+import com.ctrip.framework.apollo.openapi.dto.OpenItemDTO;
+import com.google.common.primitives.Ints;
+import org.apache.shardingsphere.orchestration.center.configuration.InstanceConfiguration;
+
+import java.util.Objects;
+import java.util.Properties;
+
+/**
+ * Apollo open api client wrapper.
+ *
+ * @author dongzonglei
+ */
+public final class ApolloOpenApiWrapper {
+    
+    private ApolloOpenApiClient client;
+    
+    private String namespace;
+    
+    private String appId;
+    
+    private String env;
+    
+    private String clusterName;
+    
+    private String administrator;
+    
+    public ApolloOpenApiWrapper(final InstanceConfiguration config, final Properties properties) {
 
 Review comment:
   The same to above.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services