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 2022/09/14 13:18:14 UTC

[GitHub] [shardingsphere] caqhy opened a new pull request, #20984: Support Nacos in cluster mode to persist metadata

caqhy opened a new pull request, #20984:
URL: https://github.com/apache/shardingsphere/pull/20984

   Fixes #ISSUSE_ID.
   
   Changes proposed in this pull request:
     - Add NacosRepository in cluster mode
     - Fix NPE when etcd listener triggers
     - Refactor example
   
   ---
   
   Before committing this PR, I'm sure that I have checked the following options:
   - [x] My code follows the [code of conduct](https://shardingsphere.apache.org/community/en/involved/conduct/code/) of this project.
   - [x] I have self-reviewed the commit code.
   - [x] I have triggered maven check: `mvn clean install -B -T2C -DskipTests -Dmaven.javadoc.skip=true -e`.
   - [x] I have made corresponding changes to the documentation.
   - [x] I have added corresponding unit tests for my changes.
   
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976362088


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);

Review Comment:
   Okay, I've refactored this 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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976357351


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;

Review Comment:
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;
+    }
+    
+    private List<KeyValue> build(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        if (findExisted(key, RegisterMetadata.PERSISTENT.isEphemeral()).isEmpty()) {
+            Instance instance = new Instance();
+            instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+            instance.setPort(RegisterMetadata.PERSISTENT.getPort());
+            instance.setEphemeral(false);
+            Map<String, String> metadataMap = new HashMap<>(2, 1);
+            metadataMap.put(key, MetadataUtil.EMPTY);
+            metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+            instance.setMetadata(metadataMap);
+            client.registerInstance(RegisterMetadata.PERSISTENT.name(), instance);
+            keyValues.add(new KeyValue(key, MetadataUtil.EMPTY, false));
+        }
+        return keyValues;

Review Comment:
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976361343


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   Okay, I've fix it



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] zhaojinchao95 commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
zhaojinchao95 commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r978837161


##########
examples/shardingsphere-jdbc-example/single-feature-example/cluster-mode-example/cluster-mode-raw-jdbc-example/src/main/java/org/apache/shardingsphere/example/cluster/mode/raw/jdbc/config/type/RepositoryType.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.example.cluster.mode.raw.jdbc.config.type;
+
+/**
+ * Repository type.
+ */
+public class RepositoryType {

Review Comment:
   add final modifier



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976358743


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {

Review Comment:
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;

Review Comment:
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976355674


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   Okay, I've fix it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   Okay, I've fix it



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976359161


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;
+    }
+    
+    /**
+     * Update preInstances.
+     * @param instances instances

Review Comment:
   OK.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data
+@RequiredArgsConstructor
+public class WatchData {

Review Comment:
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976362970


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976362088


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);

Review Comment:
   Okay, I've refactored this 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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] codecov-commenter commented on pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#issuecomment-1246961008

   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/20984?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#20984](https://codecov.io/gh/apache/shardingsphere/pull/20984?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (df8f593) into [master](https://codecov.io/gh/apache/shardingsphere/commit/1071f48e6654c5cfafca4e1d34b21c5bd8331ebe?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1071f48) will **increase** coverage by `0.11%`.
   > The diff coverage is `76.85%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #20984      +/-   ##
   ============================================
   + Coverage     61.80%   61.91%   +0.11%     
   - Complexity     2468     2549      +81     
   ============================================
     Files          4041     4057      +16     
     Lines         55486    55940     +454     
     Branches       9403     9458      +55     
   ============================================
   + Hits          34294    34637     +343     
   - Misses        18319    18397      +78     
   - Partials       2873     2906      +33     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/20984?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pository/cluster/nacos/constant/JdbcConstants.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9jb25zdGFudC9KZGJjQ29uc3RhbnRzLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...e/mode/repository/cluster/etcd/EtcdRepository.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LWV0Y2Qvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL21vZGUvcmVwb3NpdG9yeS9jbHVzdGVyL2V0Y2QvRXRjZFJlcG9zaXRvcnkuamF2YQ==) | `64.70% <41.66%> (-4.94%)` | :arrow_down: |
   | [...e/repository/cluster/nacos/utils/MetadataUtil.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy91dGlscy9NZXRhZGF0YVV0aWwuamF2YQ==) | `63.63% <63.63%> (ø)` | |
   | [...pository/cluster/nacos/lock/NacosInternalLock.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9sb2NrL05hY29zSW50ZXJuYWxMb2NrLmphdmE=) | `73.52% <73.52%> (ø)` | |
   | [...ry/cluster/nacos/listener/NamingEventListener.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9saXN0ZW5lci9OYW1pbmdFdmVudExpc3RlbmVyLmphdmE=) | `74.50% <74.50%> (ø)` | |
   | [...acos/props/metadata/DataSourceMetaDataFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9wcm9wcy9tZXRhZGF0YS9EYXRhU291cmNlTWV0YURhdGFGYWN0b3J5LmphdmE=) | `75.00% <75.00%> (ø)` | |
   | [...mode/repository/cluster/nacos/NacosRepository.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9OYWNvc1JlcG9zaXRvcnkuamF2YQ==) | `76.34% <76.34%> (ø)` | |
   | [...repository/cluster/nacos/utils/ReflectionUtil.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy91dGlscy9SZWZsZWN0aW9uVXRpbC5qYXZh) | `85.71% <85.71%> (ø)` | |
   | [...ository/cluster/nacos/entity/RegisterMetadata.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9lbnRpdHkvUmVnaXN0ZXJNZXRhZGF0YS5qYXZh) | `100.00% <100.00%> (ø)` | |
   | [...ry/cluster/nacos/lock/NacosInternalLockHolder.java](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtbW9kZS9zaGFyZGluZ3NwaGVyZS1tb2RlLXR5cGUvc2hhcmRpbmdzcGhlcmUtY2x1c3Rlci1tb2RlL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5L3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LXByb3ZpZGVyL3NoYXJkaW5nc3BoZXJlLWNsdXN0ZXItbW9kZS1yZXBvc2l0b3J5LW5hY29zL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9tb2RlL3JlcG9zaXRvcnkvY2x1c3Rlci9uYWNvcy9sb2NrL05hY29zSW50ZXJuYWxMb2NrSG9sZGVyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | ... and [17 more](https://codecov.io/gh/apache/shardingsphere/pull/20984/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976359161


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;
+    }
+    
+    /**
+     * Update preInstances.
+     * @param instances instances

Review Comment:
   OK.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;
+    }
+    
+    /**
+     * Update preInstances.
+     * @param instances instances

Review Comment:
   > Please keep blank line between java doc and @param
   
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976355260


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   Okay, I've fix it



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   Okay, I've fix it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] zhaojinchao95 commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
zhaojinchao95 commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r978838435


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/test/java/org/apache/shardingsphere/mode/repository/cluster/nacos/props/NacosPropertiesTest.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.mode.repository.cluster.nacos.props;
+
+import org.apache.shardingsphere.infra.instance.utils.IpUtils;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class NacosPropertiesTest {

Review Comment:
   Add final modifier



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] terrymanu merged pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
terrymanu merged PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976357408


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;
+    }
+    
+    private List<KeyValue> build(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        if (findExisted(key, RegisterMetadata.PERSISTENT.isEphemeral()).isEmpty()) {
+            Instance instance = new Instance();
+            instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+            instance.setPort(RegisterMetadata.PERSISTENT.getPort());
+            instance.setEphemeral(false);
+            Map<String, String> metadataMap = new HashMap<>(2, 1);
+            metadataMap.put(key, MetadataUtil.EMPTY);
+            metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+            instance.setMetadata(metadataMap);
+            client.registerInstance(RegisterMetadata.PERSISTENT.name(), instance);
+            keyValues.add(new KeyValue(key, MetadataUtil.EMPTY, false));
+        }
+        return keyValues;

Review Comment:
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;
+    }
+    
+    private List<KeyValue> build(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        if (findExisted(key, RegisterMetadata.PERSISTENT.isEphemeral()).isEmpty()) {
+            Instance instance = new Instance();
+            instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+            instance.setPort(RegisterMetadata.PERSISTENT.getPort());
+            instance.setEphemeral(false);
+            Map<String, String> metadataMap = new HashMap<>(2, 1);
+            metadataMap.put(key, MetadataUtil.EMPTY);
+            metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+            instance.setMetadata(metadataMap);
+            client.registerInstance(RegisterMetadata.PERSISTENT.name(), instance);
+            keyValues.add(new KeyValue(key, MetadataUtil.EMPTY, false));
+        }
+        return keyValues;

Review Comment:
   > Please name as `result` for return value
   
   OKOK



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r978263165


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;
+    }
+    
+    private List<KeyValue> build(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        if (findExisted(key, RegisterMetadata.PERSISTENT.isEphemeral()).isEmpty()) {
+            Instance instance = new Instance();
+            instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+            instance.setPort(RegisterMetadata.PERSISTENT.getPort());
+            instance.setEphemeral(false);
+            Map<String, String> metadataMap = new HashMap<>(2, 1);
+            metadataMap.put(key, MetadataUtil.EMPTY);
+            metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+            instance.setMetadata(metadataMap);
+            client.registerInstance(RegisterMetadata.PERSISTENT.name(), instance);
+            keyValues.add(new KeyValue(key, MetadataUtil.EMPTY, false));
+        }
+        return keyValues;

Review Comment:
   > Please name as `result` for return value
   
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;

Review Comment:
   You're right. I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976358869


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;

Review Comment:
   > Please name as `result` for return value
   
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r978263225


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;

Review Comment:
   > How about use Constructor to init the fields?
   
   You're right. I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;
+    
+    @Setter
+    @Getter
+    private NamingEventListener listener;
+    
+    @Getter
+    private final boolean ephemeral;
+    
+    /**
+     * Find corresponding registerMetadata.
+     *
+     * @param isEphemeral isEphemeral
+     * @return registerMetadata
+     */
+    public static RegisterMetadata of(final boolean isEphemeral) {
+        return Stream.of(values())
+                .filter(registerMetadata -> registerMetadata.ephemeral == isEphemeral).findAny()
+                .orElseThrow(() -> new IllegalArgumentException("Status not exist: " + isEphemeral));
+    }
+    
+    /**
+     * This is to ensure that port is different from the last time when persisting.
+     *
+     * @return fake port
+     */
+    public int getPort() {
+        int port = this.port.incrementAndGet();
+        if (port == Integer.MIN_VALUE) {
+            throw new IllegalStateException("Specified cluster ip exceeded the maximum number of persisting");
+        }
+        return port;

Review Comment:
   OK.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;
+    
+    @Setter
+    @Getter
+    private NamingEventListener listener;
+    
+    @Getter
+    private final boolean ephemeral;
+    
+    /**
+     * Find corresponding registerMetadata.
+     *
+     * @param isEphemeral isEphemeral
+     * @return registerMetadata
+     */
+    public static RegisterMetadata of(final boolean isEphemeral) {
+        return Stream.of(values())
+                .filter(registerMetadata -> registerMetadata.ephemeral == isEphemeral).findAny()
+                .orElseThrow(() -> new IllegalArgumentException("Status not exist: " + isEphemeral));
+    }
+    
+    /**
+     * This is to ensure that port is different from the last time when persisting.
+     *
+     * @return fake port
+     */
+    public int getPort() {
+        int port = this.port.incrementAndGet();
+        if (port == Integer.MIN_VALUE) {
+            throw new IllegalStateException("Specified cluster ip exceeded the maximum number of persisting");
+        }
+        return port;

Review Comment:
   > Please name as `result` for return value
   
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976353619


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();

Review Comment:
   > What does `preInstances` mean?
   
   @terrymanu 
   `preInstances` is used to cache the last callback data.Compare it with this callback data to get data change type.
   ![1663761331300](https://user-images.githubusercontent.com/75082574/191497594-7cd184e2-0384-48f9-ac87-4d97c9f3d207.png)
   



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976353619


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();

Review Comment:
   @terrymanu 
   `preInstances` is used to cache the last callback data.
   Compare it with this callback data to get data change type.
   ![1663761331300](https://user-images.githubusercontent.com/75082574/191497594-7cd184e2-0384-48f9-ac87-4d97c9f3d207.png)
   



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] terrymanu commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
terrymanu commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r975937570


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   Is it necessary to use @Data here? 



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;

Review Comment:
   Please name as `result` for return value



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data
+@RequiredArgsConstructor
+public class WatchData {

Review Comment:
   Please add final if class does not design for extension



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;
+    
+    @Setter
+    @Getter
+    private NamingEventListener listener;
+    
+    @Getter
+    private final boolean ephemeral;
+    
+    /**
+     * Find corresponding registerMetadata.
+     *
+     * @param isEphemeral isEphemeral
+     * @return registerMetadata
+     */
+    public static RegisterMetadata of(final boolean isEphemeral) {
+        return Stream.of(values())
+                .filter(registerMetadata -> registerMetadata.ephemeral == isEphemeral).findAny()
+                .orElseThrow(() -> new IllegalArgumentException("Status not exist: " + isEphemeral));
+    }
+    
+    /**
+     * This is to ensure that port is different from the last time when persisting.
+     *
+     * @return fake port
+     */
+    public int getPort() {
+        int port = this.port.incrementAndGet();
+        if (port == Integer.MIN_VALUE) {
+            throw new IllegalStateException("Specified cluster ip exceeded the maximum number of persisting");
+        }
+        return port;

Review Comment:
   Please name as `result` for return value



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   Please add final if class does not design for extension



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();

Review Comment:
   What does `preInstances` mean?



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Please add private constructor for util class



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {

Review Comment:
   Please add final if class does not design for extension



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);

Review Comment:
   How about just use boolean?



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;
+    }
+    
+    private List<KeyValue> build(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        if (findExisted(key, RegisterMetadata.PERSISTENT.isEphemeral()).isEmpty()) {
+            Instance instance = new Instance();
+            instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+            instance.setPort(RegisterMetadata.PERSISTENT.getPort());
+            instance.setEphemeral(false);
+            Map<String, String> metadataMap = new HashMap<>(2, 1);
+            metadataMap.put(key, MetadataUtil.EMPTY);
+            metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+            instance.setMetadata(metadataMap);
+            client.registerInstance(RegisterMetadata.PERSISTENT.name(), instance);
+            keyValues.add(new KeyValue(key, MetadataUtil.EMPTY, false));
+        }
+        return keyValues;

Review Comment:
   Please name as `result` for return value



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data

Review Comment:
   Is it necessary to use @DaTa here?



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;

Review Comment:
   How about use Constructor to init the fields?



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;

Review Comment:
   Please name as `result` for return value



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Please add final if class does not design for extension



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();
+    
+    private final Map<String, DataChangedEventListener> prefixListenerMap;
+    
+    @Override
+    public void onEvent(final Event event) {
+        if (event instanceof NamingEvent) {
+            NamingEvent namingEvent = (NamingEvent) event;
+            List<Instance> instances = namingEvent.getInstances().stream().sorted(Comparator.comparing(MetadataUtil::getKey)).collect(Collectors.toList());
+            List<WatchData> watchDataList = new LinkedList<>();
+            instances.forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    Instance preInstance = preInstances.remove(key);
+                    WatchData watchData = new WatchData(key, preInstance, instance, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            preInstances.values().stream().sorted(Comparator.comparing(MetadataUtil::getKey).reversed()).forEach(instance -> prefixListenerMap.forEach((prefixPath, listener) -> {
+                String key = MetadataUtil.getKey(instance);
+                if (key.startsWith(prefixPath)) {
+                    WatchData watchData = new WatchData(key, instance, null, listener);
+                    watchDataList.add(watchData);
+                }
+            }));
+            watchDataList.forEach(watchData -> {
+                String key = watchData.getKey();
+                Instance preInstance = watchData.getPreInstance();
+                Instance instance = watchData.getInstance();
+                DataChangedEventListener listener = watchData.getListener();
+                DataChangedEvent.Type changedType = getEventChangedType(preInstance, instance);
+                switch (changedType) {
+                    case ADDED:
+                    case UPDATED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(instance), changedType));
+                        break;
+                    case DELETED:
+                        listener.onChange(new DataChangedEvent(key, MetadataUtil.getValue(preInstance), changedType));
+                        break;
+                    default:
+                }
+            });
+            setPreInstances(instances);
+        }
+    }
+    
+    private DataChangedEvent.Type getEventChangedType(final Instance preInstance, final Instance instance) {
+        DataChangedEvent.Type type;
+        if (Objects.isNull(preInstance) && Objects.nonNull(instance)) {
+            type = DataChangedEvent.Type.ADDED;
+        } else if (Objects.nonNull(preInstance) && Objects.nonNull(instance)
+                && MetadataUtil.getTimestamp(preInstance) != MetadataUtil.getTimestamp(instance)) {
+            type = DataChangedEvent.Type.UPDATED;
+        } else if (Objects.nonNull(preInstance) && Objects.isNull(instance)) {
+            type = DataChangedEvent.Type.DELETED;
+        } else {
+            return DataChangedEvent.Type.IGNORED;
+        }
+        return type;
+    }
+    
+    /**
+     * Update preInstances.
+     * @param instances instances

Review Comment:
   Please keep blank line between java doc and @param



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976358518


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;

Review Comment:
   You're right. I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);
+    
+    @Setter
+    private AtomicInteger port;
+    
+    @Setter
+    @Getter
+    private NamingEventListener listener;
+    
+    @Getter
+    private final boolean ephemeral;
+    
+    /**
+     * Find corresponding registerMetadata.
+     *
+     * @param isEphemeral isEphemeral
+     * @return registerMetadata
+     */
+    public static RegisterMetadata of(final boolean isEphemeral) {
+        return Stream.of(values())
+                .filter(registerMetadata -> registerMetadata.ephemeral == isEphemeral).findAny()
+                .orElseThrow(() -> new IllegalArgumentException("Status not exist: " + isEphemeral));
+    }
+    
+    /**
+     * This is to ensure that port is different from the last time when persisting.
+     *
+     * @return fake port
+     */
+    public int getPort() {
+        int port = this.port.incrementAndGet();
+        if (port == Integer.MIN_VALUE) {
+            throw new IllegalStateException("Specified cluster ip exceeded the maximum number of persisting");
+        }
+        return port;

Review Comment:
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976355260


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   Okay, I've fix it



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r978262793


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data

Review Comment:
   > Is it necessary to use @DaTa here?
   
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   > Please add final if class does not design for extension
   
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
sandynz commented on PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#issuecomment-1346095510

   Hi @caqhy , could you supply some screenshots after metadata has been persisted into Nacos? e.g. view `rules` or `tables/tableName` content.
   
   Some user feedback newline in table metadata is shown wrongly, newline is shown as `\n`.
   Check [screenshot]( https://user-images.githubusercontent.com/29478697/206977047-ee33908b-4673-4752-911f-203765a00486.png ) for details.
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976357351


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;

Review Comment:
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.mode.repository.cluster.nacos;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.NamingFactory;
+import com.alibaba.nacos.api.naming.NamingService;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import com.alibaba.nacos.common.utils.CollectionUtils;
+import com.alibaba.nacos.common.utils.StringUtils;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryException;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.RegisterMetadata;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosProperties;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.props.NacosPropertyKey;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Registry repository of Nacos.
+ */
+@Slf4j
+public final class NacosRepository implements ClusterPersistRepository {
+    
+    private NamingService client;
+    
+    private NacosProperties nacosProps;
+    
+    @Override
+    public void init(final ClusterPersistRepositoryConfiguration config) {
+        nacosProps = new NacosProperties(config.getProps());
+        initClient(config);
+        initRegisterMetadata();
+    }
+    
+    private void initClient(final ClusterPersistRepositoryConfiguration config) {
+        Properties props = new Properties();
+        props.setProperty("serverAddr", config.getServerLists());
+        props.setProperty("namespace", config.getNamespace());
+        try {
+            client = NamingFactory.createNamingService(props);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void initRegisterMetadata() {
+        try {
+            String ip = nacosProps.getValue(NacosPropertyKey.CLUSTER_IP);
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                AtomicInteger port = client.getAllInstances(registerMetadata.name(), false).stream()
+                        .filter(filterInstance -> StringUtils.equals(filterInstance.getIp(), ip)).max(Comparator.comparing(Instance::getPort))
+                        .map(convert -> new AtomicInteger(convert.getPort())).orElse(new AtomicInteger(Integer.MIN_VALUE));
+                registerMetadata.setPort(port);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistEphemeral(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            if (!findExisted(key, true).isEmpty()) {
+                delete(key);
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persistExclusiveEphemeral(final String key, final String value) {
+        try {
+            if (!findExisted(key, true).isEmpty()) {
+                throw new IllegalStateException("Key: " + key + " already exists");
+            }
+            put(key, value, true);
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public boolean persistLock(final String lockKey, final long timeoutMillis) {
+        // TODO
+        return false;
+    }
+    
+    @Override
+    public void deleteLock(final String lockKey) {
+        // TODO
+    }
+    
+    @Override
+    public void watch(final String key, final DataChangedEventListener listener) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                NamingEventListener eventListener = registerMetadata.getListener();
+                if (Objects.isNull(eventListener)) {
+                    Map<String, DataChangedEventListener> parentPathListenerMap = new HashMap<>();
+                    parentPathListenerMap.put(key, listener);
+                    eventListener = new NamingEventListener(parentPathListenerMap);
+                    registerMetadata.setListener(eventListener);
+                    client.subscribe(registerMetadata.name(), eventListener);
+                } else {
+                    eventListener.put(key, listener);
+                }
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public String get(final String key) {
+        try {
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Optional<Instance> optional = findExisted(key, registerMetadata.isEphemeral()).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+                if (optional.isPresent()) {
+                    return MetadataUtil.getValue(optional.get());
+                }
+            }
+            return null;
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public List<String> getChildrenKeys(final String key) {
+        try {
+            Stream<String> concatKeys = Stream.empty();
+            for (RegisterMetadata registerMetadata : RegisterMetadata.values()) {
+                Stream<String> keys = findExisted(registerMetadata.isEphemeral()).stream()
+                        .map(instance -> {
+                            String fullPath = MetadataUtil.getKey(instance);
+                            if (fullPath.startsWith(key + PATH_SEPARATOR)) {
+                                String pathWithoutPrefix = fullPath.substring((key + PATH_SEPARATOR).length());
+                                return pathWithoutPrefix.contains(PATH_SEPARATOR) ? pathWithoutPrefix.substring(0, pathWithoutPrefix.indexOf(PATH_SEPARATOR)) : pathWithoutPrefix;
+                            }
+                            return null;
+                        }).filter(Objects::nonNull);
+                concatKeys = Stream.concat(concatKeys, keys);
+            }
+            return concatKeys.distinct().sorted(Comparator.reverseOrder()).collect(Collectors.toList());
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    @Override
+    public void persist(final String key, final String value) {
+        try {
+            if (Objects.isNull(value)) {
+                throw new IllegalArgumentException("Value cannot be null");
+            }
+            Optional<Instance> optional = findExisted(key, false).stream().max(Comparator.comparing(MetadataUtil::getTimestamp));
+            if (optional.isPresent()) {
+                update(optional.get(), value);
+            } else {
+                put(key, value, false);
+            }
+            // CHECKSTYLE:OFF
+        } catch (Exception cause) {
+            // CHECKSTYLE:ON
+            throw new ClusterPersistRepositoryException(cause);
+        }
+    }
+    
+    private void put(final String key, final String value, final boolean ephemeral) throws NacosException, InterruptedException {
+        final List<KeyValue> keyValues = buildParentPath(key);
+        RegisterMetadata registerMetadata = RegisterMetadata.of(ephemeral);
+        Instance instance = new Instance();
+        instance.setIp(nacosProps.getValue(NacosPropertyKey.CLUSTER_IP));
+        instance.setPort(registerMetadata.getPort());
+        instance.setEphemeral(ephemeral);
+        Map<String, String> metadataMap = new HashMap<>(5, 1);
+        if (ephemeral) {
+            fillEphemeralMetadata(metadataMap);
+        }
+        metadataMap.put(key, value);
+        metadataMap.put(MetadataUtil.UTC_ZONE_OFFSET.toString(), String.valueOf(MetadataUtil.getTimestamp()));
+        instance.setMetadata(metadataMap);
+        client.registerInstance(registerMetadata.name(), instance);
+        keyValues.add(new KeyValue(key, value, ephemeral));
+        waitValue(keyValues);
+    }
+    
+    private List<KeyValue> buildParentPath(final String key) throws NacosException {
+        List<KeyValue> keyValues = new LinkedList<>();
+        StringBuilder parentPath = new StringBuilder();
+        String[] partPath = key.split(PATH_SEPARATOR);
+        for (int index = 1; index < partPath.length - 1; index++) {
+            String path = parentPath.append(PATH_SEPARATOR).append(partPath[index]).toString();
+            if (findExisted(path, false).isEmpty()) {
+                keyValues.addAll(build(path));
+            }
+        }
+        return keyValues;

Review Comment:
   > Please name as `result` for return value
   
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976361343


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data

Review Comment:
   > Is it necessary to use @DaTa here?
   
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976361204


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor

Review Comment:
   > Please use @RequiredArgsConstructor
   
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976356948


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/RegisterMetadata.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.entity;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.listener.NamingEventListener;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+
+/**
+ * Register metadata.
+ */
+@RequiredArgsConstructor
+public enum RegisterMetadata {
+    
+    /**
+     * persistent.
+     */
+    PERSISTENT(false),
+    
+    /**
+     * ephemeral.
+     */
+    EPHEMERAL(true);

Review Comment:
   I have refactor this 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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976363228


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data

Review Comment:
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] terrymanu commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
terrymanu commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r979399315


##########
shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml:
##########
@@ -93,6 +93,11 @@
             <artifactId>shardingsphere-cluster-mode-repository-etcd</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-cluster-mode-repository-nacos</artifactId>
+            <version>${project.version}</version>
+        </dependency>

Review Comment:
   I need to remove it because there is no license declare for Nacos and its dependencies



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#issuecomment-1246775707

   when etcd listener triggers in cluster mode,application throws NPE.
   ```[ERROR] 2022-09-14 21:31:54.592 [grpc-default-executor-1] c.g.common.eventbus.EventBus.default - Exception thrown by subscriber method renew(org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.rule.RuleConfigurationsChangedEvent) on subscriber org.apache.shardingsphere.mode.manager.cluster.coordinator.ClusterContextManagerCoordinator@6ea246af when dispatching event: org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.rule.RuleConfigurationsChangedEvent@36eaf53b
   java.lang.NullPointerException: null
   	at org.apache.shardingsphere.mode.manager.ContextManager.getStaleResourceHeldRules(ContextManager.java:325)
   	at org.apache.shardingsphere.mode.manager.ContextManager.alterRuleConfiguration(ContextManager.java:271)
   	at org.apache.shardingsphere.mode.manager.cluster.coordinator.ClusterContextManagerCoordinator.renew(ClusterContextManagerCoordinator.java:149)
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at com.google.common.eventbus.Subscriber.invokeSubscriberMethod(Subscriber.java:87)
   	at com.google.common.eventbus.Subscriber$SynchronizedSubscriber.invokeSubscriberMethod(Subscriber.java:144)
   	at com.google.common.eventbus.Subscriber$1.run(Subscriber.java:72)
   	at com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
   	at com.google.common.eventbus.Subscriber.dispatchEvent(Subscriber.java:67)
   	at com.google.common.eventbus.Dispatcher$PerThreadQueuedDispatcher.dispatch(Dispatcher.java:108)
   	at com.google.common.eventbus.EventBus.post(EventBus.java:212)
   	at org.apache.shardingsphere.infra.util.eventbus.EventBusContext.post(EventBusContext.java:51)
   	at java.util.Optional.ifPresent(Optional.java:159)
   	at org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceWatcherFactory.lambda$watch$0(GovernanceWatcherFactory.java:59)
   	at org.apache.shardingsphere.mode.repository.cluster.etcd.EtcdRepository.lambda$watch$2(EtcdRepository.java:157)
   	at io.etcd.jetcd.Watch$1.onNext(Watch.java:168)
   	at io.etcd.jetcd.WatchImpl$WatcherImpl.onNext(WatchImpl.java:289)
   	at io.etcd.jetcd.WatchImpl$WatcherImpl.onNext(WatchImpl.java:111)
   	at io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onMessage(ClientCalls.java:465)
   	at io.grpc.ForwardingClientCallListener.onMessage(ForwardingClientCallListener.java:33)
   	at io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInternal(ClientCallImpl.java:652)
   	at io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInContext(ClientCallImpl.java:637)
   	at io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
   	at io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976359237


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data
+@RequiredArgsConstructor
+public class WatchData {

Review Comment:
   OK.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data
+@RequiredArgsConstructor
+public class WatchData {

Review Comment:
   > Please add final if class does not design for extension
   
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/WatchData.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.Data;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+
+/**
+ * Watch data.
+ */
+@Data

Review Comment:
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976358743


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {

Review Comment:
   OK



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {

Review Comment:
   > Please add final if class does not design for extension
   
   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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976362970


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/utils/MetadataUtil.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.mode.repository.cluster.nacos.utils;
+
+import com.alibaba.nacos.api.exception.NacosException;
+import com.alibaba.nacos.api.naming.PreservedMetadataKeys;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.SneakyThrows;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+/**
+ * Metadata util.
+ */
+public class MetadataUtil {

Review Comment:
   > Please add private constructor for util class
   
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976361543


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   Okay, I've fixed it.



##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   > Please add final if class does not design for extension
   
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976361543


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/entity/KeyValue.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.repository.cluster.nacos.entity;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+/**
+ * Key value.
+ */
+@AllArgsConstructor
+@Data
+public class KeyValue {

Review Comment:
   Okay, I've fixed it.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] caqhy commented on a diff in pull request #20984: Support Nacos to persist metadata in cluster mode

Posted by GitBox <gi...@apache.org>.
caqhy commented on code in PR #20984:
URL: https://github.com/apache/shardingsphere/pull/20984#discussion_r976353619


##########
shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-repository/shardingsphere-cluster-mode-repository-provider/shardingsphere-cluster-mode-repository-nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/listener/NamingEventListener.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.mode.repository.cluster.nacos.listener;
+
+import com.alibaba.nacos.api.naming.listener.Event;
+import com.alibaba.nacos.api.naming.listener.EventListener;
+import com.alibaba.nacos.api.naming.listener.NamingEvent;
+import com.alibaba.nacos.api.naming.pojo.Instance;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
+import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
+import org.apache.shardingsphere.mode.repository.cluster.nacos.utils.MetadataUtil;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Naming event listener.
+ */
+@RequiredArgsConstructor
+public class NamingEventListener implements EventListener {
+    
+    private Map<String, Instance> preInstances = new HashMap<>();

Review Comment:
   @terrymanu 
   `preInstances` is used to cache the last callback data.
   Compare it with this callback data to get data change type.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org