You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shenyu.apache.org by xi...@apache.org on 2022/06/09 00:08:21 UTC

[incubator-shenyu] branch master updated: [type: refactor] Refactor http long polling. (#3500)

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

xiaoyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-shenyu.git


The following commit(s) were added to refs/heads/master by this push:
     new 958c8efc3 [type: refactor] Refactor http long polling. (#3500)
958c8efc3 is described below

commit 958c8efc31d28fa6f8c3c71b69103184df16ed83
Author: Kevin Clair <70...@qq.com>
AuthorDate: Thu Jun 9 08:08:12 2022 +0800

    [type: refactor] Refactor http long polling. (#3500)
    
    * change init.
    
    * Add constant.
    
    * Add property to record the expired time of token.
    
    * Add token manager in http long polling client.
    
    * Http long polling refactor.
    
    * code optimize.
    
    * delete todo.
    
    * Add test case.
    
    * optimize test case.
    
    * code clean.
    
    * add bean register.
    
    * reset profile.
    
    * reset file.
    
    * check style.
---
 .../admin/model/vo/LoginDashboardUserVO.java       |  25 ++++
 .../service/impl/DashboardUserServiceImpl.java     |   2 +-
 shenyu-bootstrap/pom.xml                           | 108 +++++++---------
 .../src/main/resources/application.yml             |   2 +
 .../apache/shenyu/common/constant/Constants.java   |   5 +
 .../sync/data/http/HttpSyncDataConfiguration.java  |  67 +++++++---
 .../http/HttpClientPluginConfigurationTest.java    |   3 +-
 .../shenyu/sync/data/http/AccessTokenManager.java  | 137 +++++++++++++++++++++
 .../shenyu/sync/data/http/HttpSyncDataService.java |  88 ++++---------
 .../shenyu/sync/data/http/config/HttpConfig.java   |   3 +
 .../data/http/refresh/AbstractDataRefresh.java     |   6 -
 .../sync/data/http/refresh/AppAuthDataRefresh.java |   3 +-
 .../sync/data/http/refresh/MetaDataRefresh.java    |   6 +-
 .../sync/data/http/refresh/PluginDataRefresh.java  |   3 +-
 .../sync/data/http/refresh/RuleDataRefresh.java    |   6 +-
 .../data/http/refresh/SelectorDataRefresh.java     |   3 +-
 .../sync/data/http/AccessTokenManagerTest.java     | 120 ++++++++++++++++++
 .../sync/data/http/HttpSyncDataServiceTest.java    |  22 +++-
 18 files changed, 446 insertions(+), 163 deletions(-)

diff --git a/shenyu-admin/src/main/java/org/apache/shenyu/admin/model/vo/LoginDashboardUserVO.java b/shenyu-admin/src/main/java/org/apache/shenyu/admin/model/vo/LoginDashboardUserVO.java
index a589f0e0b..ec45a96b5 100644
--- a/shenyu-admin/src/main/java/org/apache/shenyu/admin/model/vo/LoginDashboardUserVO.java
+++ b/shenyu-admin/src/main/java/org/apache/shenyu/admin/model/vo/LoginDashboardUserVO.java
@@ -31,6 +31,11 @@ public class LoginDashboardUserVO extends DashboardUserVO {
      */
     private String token;
 
+    /**
+     * expired time(milliSeconds).
+     */
+    private Long expiredTime;
+
     public LoginDashboardUserVO() {
     }
 
@@ -54,6 +59,26 @@ public class LoginDashboardUserVO extends DashboardUserVO {
         return this;
     }
 
+    /**
+     * Gets the value of expiredTime.
+     *
+     * @return the value of expiredTime
+     */
+    public Long getExpiredTime() {
+        return expiredTime;
+    }
+
+    /**
+     * Sets the expiredTime.
+     *
+     * @param expiredTime expiredTime
+     * @return {@link LoginDashboardUserVO}
+     */
+    public LoginDashboardUserVO setExpiredTime(final Long expiredTime) {
+        this.expiredTime = expiredTime;
+        return this;
+    }
+
     /**
      * build loginDashboardUserVO.
      *
diff --git a/shenyu-admin/src/main/java/org/apache/shenyu/admin/service/impl/DashboardUserServiceImpl.java b/shenyu-admin/src/main/java/org/apache/shenyu/admin/service/impl/DashboardUserServiceImpl.java
index 96069c8da..a22ae2618 100644
--- a/shenyu-admin/src/main/java/org/apache/shenyu/admin/service/impl/DashboardUserServiceImpl.java
+++ b/shenyu-admin/src/main/java/org/apache/shenyu/admin/service/impl/DashboardUserServiceImpl.java
@@ -239,7 +239,7 @@ public class DashboardUserServiceImpl implements DashboardUserService {
                 return loginUser;
             }
             return loginUser.setToken(JwtUtils.generateToken(finalDashboardUserVO.getUserName(), finalDashboardUserVO.getPassword(),
-                    jwtProperties.getExpiredSeconds()));
+                    jwtProperties.getExpiredSeconds())).setExpiredTime(jwtProperties.getExpiredSeconds());
         }).orElse(null);
     }
 
diff --git a/shenyu-bootstrap/pom.xml b/shenyu-bootstrap/pom.xml
index 25ee710cd..d1134ae82 100644
--- a/shenyu-bootstrap/pom.xml
+++ b/shenyu-bootstrap/pom.xml
@@ -28,7 +28,6 @@
     <properties>
         <nacos-discovery.version>2021.0.1.0</nacos-discovery.version>
         <eureka-client.version>3.1.2</eureka-client.version>
-        <bootstrap-curator.version>4.0.1</bootstrap-curator.version>
     </properties>
 
     <dependencies>
@@ -244,7 +243,7 @@
         <!--shenyu sofa plugin end-->
 
         <!--shenyu springCloud plugin start-->
-       <dependency>
+        <dependency>
             <groupId>org.apache.shenyu</groupId>
             <artifactId>shenyu-spring-boot-starter-plugin-springcloud</artifactId>
             <version>${project.version}</version>
@@ -273,9 +272,9 @@
             <version>${spring-cloud-netflix-ribbon.version}</version>
         </dependency>
         <!--shenyu springCloud plugin start-->
-        
+
         <!--shenyu  apache dubbo plugin start-->
-         <dependency>
+        <dependency>
             <groupId>org.apache.shenyu</groupId>
             <artifactId>shenyu-spring-boot-starter-plugin-apache-dubbo</artifactId>
             <version>${project.version}</version>
@@ -286,52 +285,52 @@
             <version>2.7.15</version>
         </dependency>
         <!-- Dubbo Nacos registry dependency -->
-      <!--  <dependency>
-            <groupId>org.apache.dubbo</groupId>
-            <artifactId>dubbo-registry-nacos</artifactId>
-            <version>2.7.15</version>
-        </dependency>
-        <dependency>
-            <groupId>com.alibaba.nacos</groupId>
-            <artifactId>nacos-client</artifactId>
-            <version>1.1.4</version>
-        </dependency>-->
+        <!--  <dependency>
+              <groupId>org.apache.dubbo</groupId>
+              <artifactId>dubbo-registry-nacos</artifactId>
+              <version>2.7.15</version>
+          </dependency>
+          <dependency>
+              <groupId>com.alibaba.nacos</groupId>
+              <artifactId>nacos-client</artifactId>
+              <version>1.1.4</version>
+          </dependency>-->
         <!-- Dubbo zookeeper registry dependency start -->
-       <!-- Dubbo zookeeper registry dependency end -->
+        <!-- Dubbo zookeeper registry dependency end -->
         <!-- shenyu  apache dubbo plugin end-->
 
         <!--shenyu alibaba dubbo plugin start-->
-<!--        <dependency>-->
-<!--            <groupId>org.apache.shenyu</groupId>-->
-<!--            <artifactId>shenyu-spring-boot-starter-plugin-alibaba-dubbo</artifactId>-->
-<!--            <version>${project.version}</version>-->
-<!--        </dependency>-->
-<!--        <dependency>-->
-<!--            <groupId>com.alibaba</groupId>-->
-<!--            <artifactId>dubbo</artifactId>-->
-<!--            <version>${alibaba.dubbo.version}</version>-->
-<!--        </dependency>-->
-<!--        <dependency>-->
-<!--            <groupId>org.apache.curator</groupId>-->
-<!--            <artifactId>curator-client</artifactId>-->
-<!--            <version>${curator.version}</version>-->
-<!--            <exclusions>-->
-<!--                <exclusion>-->
-<!--                    <artifactId>log4j</artifactId>-->
-<!--                    <groupId>log4j</groupId>-->
-<!--                </exclusion>-->
-<!--            </exclusions>-->
-<!--        </dependency>-->
-<!--        <dependency>-->
-<!--            <groupId>org.apache.curator</groupId>-->
-<!--            <artifactId>curator-framework</artifactId>-->
-<!--            <version>${curator.version}</version>-->
-<!--        </dependency>-->
-<!--        <dependency>-->
-<!--            <groupId>org.apache.curator</groupId>-->
-<!--            <artifactId>curator-recipes</artifactId>-->
-<!--            <version>${curator.version}</version>-->
-<!--        </dependency>-->
+        <!--        <dependency>-->
+        <!--            <groupId>org.apache.shenyu</groupId>-->
+        <!--            <artifactId>shenyu-spring-boot-starter-plugin-alibaba-dubbo</artifactId>-->
+        <!--            <version>${project.version}</version>-->
+        <!--        </dependency>-->
+        <!--        <dependency>-->
+        <!--            <groupId>com.alibaba</groupId>-->
+        <!--            <artifactId>dubbo</artifactId>-->
+        <!--            <version>${alibaba.dubbo.version}</version>-->
+        <!--        </dependency>-->
+        <!--        <dependency>-->
+        <!--            <groupId>org.apache.curator</groupId>-->
+        <!--            <artifactId>curator-client</artifactId>-->
+        <!--            <version>${curator.version}</version>-->
+        <!--            <exclusions>-->
+        <!--                <exclusion>-->
+        <!--                    <artifactId>log4j</artifactId>-->
+        <!--                    <groupId>log4j</groupId>-->
+        <!--                </exclusion>-->
+        <!--            </exclusions>-->
+        <!--        </dependency>-->
+        <!--        <dependency>-->
+        <!--            <groupId>org.apache.curator</groupId>-->
+        <!--            <artifactId>curator-framework</artifactId>-->
+        <!--            <version>${curator.version}</version>-->
+        <!--        </dependency>-->
+        <!--        <dependency>-->
+        <!--            <groupId>org.apache.curator</groupId>-->
+        <!--            <artifactId>curator-recipes</artifactId>-->
+        <!--            <version>${curator.version}</version>-->
+        <!--        </dependency>-->
         <!--shenyu alibaba dubbo plugin end-->
 
         <!--shenyu spring cloud alibaba dubbo plugin start-->
@@ -456,23 +455,6 @@
         </dependency>
         <!--shenyu logging-rocketmq plugin end-->
 
-        <!-- curator start -->
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-client</artifactId>
-            <version>${bootstrap-curator.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-framework</artifactId>
-            <version>${bootstrap-curator.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-recipes</artifactId>
-            <version>${bootstrap-curator.version}</version>
-        </dependency>
-        <!-- curator end -->
     </dependencies>
     <profiles>
         <profile>
diff --git a/shenyu-bootstrap/src/main/resources/application.yml b/shenyu-bootstrap/src/main/resources/application.yml
index 616f5712f..f86cd8964 100644
--- a/shenyu-bootstrap/src/main/resources/application.yml
+++ b/shenyu-bootstrap/src/main/resources/application.yml
@@ -173,6 +173,8 @@ shenyu:
 #      connectionTimeout: 2000
 #    http:
 #      url: http://localhost:9095
+#      username:
+#      password:
 #    nacos:
 #      url: localhost:8848
 #      namespace: 1c10d748-af86-43b9-8265-75f487d20c6c
diff --git a/shenyu-common/src/main/java/org/apache/shenyu/common/constant/Constants.java b/shenyu-common/src/main/java/org/apache/shenyu/common/constant/Constants.java
index bf79ca595..ac106c868 100644
--- a/shenyu-common/src/main/java/org/apache/shenyu/common/constant/Constants.java
+++ b/shenyu-common/src/main/java/org/apache/shenyu/common/constant/Constants.java
@@ -567,6 +567,11 @@ public interface Constants {
      */
     String ADMIN_RESULT_TOKEN = "token";
 
+    /**
+     * The admin return result expired time.
+     */
+    String ADMIN_RESULT_EXPIRED_TIME = "expiredTime";
+
     /**
      * The admin userName.
      */
diff --git a/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/main/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpSyncDataConfiguration.java b/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/main/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpSyncDataConfiguration.java
index afcfa2194..3194ceb5b 100644
--- a/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/main/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpSyncDataConfiguration.java
+++ b/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/main/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpSyncDataConfiguration.java
@@ -17,10 +17,12 @@
 
 package org.apache.shenyu.springboot.starter.sync.data.http;
 
+import org.apache.shenyu.common.constant.HttpConstants;
 import org.apache.shenyu.sync.data.api.AuthDataSubscriber;
 import org.apache.shenyu.sync.data.api.MetaDataSubscriber;
 import org.apache.shenyu.sync.data.api.PluginDataSubscriber;
 import org.apache.shenyu.sync.data.api.SyncDataService;
+import org.apache.shenyu.sync.data.http.AccessTokenManager;
 import org.apache.shenyu.sync.data.http.HttpSyncDataService;
 import org.apache.shenyu.sync.data.http.config.HttpConfig;
 import org.slf4j.Logger;
@@ -28,9 +30,11 @@ import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.ObjectProvider;
 import org.springframework.boot.autoconfigure.condition.ConditionalOnClass;
 import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
-import org.springframework.boot.context.properties.ConfigurationProperties;
+import org.springframework.boot.context.properties.EnableConfigurationProperties;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
+import org.springframework.http.client.OkHttp3ClientHttpRequestFactory;
+import org.springframework.web.client.RestTemplate;
 
 import java.util.Collections;
 import java.util.List;
@@ -42,35 +46,64 @@ import java.util.Objects;
 @Configuration
 @ConditionalOnClass(HttpSyncDataService.class)
 @ConditionalOnProperty(prefix = "shenyu.sync.http", name = "url")
+@EnableConfigurationProperties(value = HttpConfig.class)
 public class HttpSyncDataConfiguration {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(HttpSyncDataConfiguration.class);
 
     /**
-     * Http sync data service.
+     * Rest template.
      *
-     * @param httpConfig        the http config
-     * @param pluginSubscriber the plugin subscriber
-     * @param metaSubscribers   the meta subscribers
-     * @param authSubscribers   the auth subscribers
-     * @return the sync data service
+     * @param httpConfig the http config
+     * @return the rest template
      */
     @Bean
-    public SyncDataService httpSyncDataService(final ObjectProvider<HttpConfig> httpConfig, final ObjectProvider<PluginDataSubscriber> pluginSubscriber,
-                                           final ObjectProvider<List<MetaDataSubscriber>> metaSubscribers, final ObjectProvider<List<AuthDataSubscriber>> authSubscribers) {
-        LOGGER.info("you use http long pull sync shenyu data");
-        return new HttpSyncDataService(Objects.requireNonNull(httpConfig.getIfAvailable()), Objects.requireNonNull(pluginSubscriber.getIfAvailable()),
-                metaSubscribers.getIfAvailable(Collections::emptyList), authSubscribers.getIfAvailable(Collections::emptyList));
+    public RestTemplate restTemplate(final HttpConfig httpConfig) {
+        OkHttp3ClientHttpRequestFactory factory = new OkHttp3ClientHttpRequestFactory();
+        factory.setConnectTimeout(Objects.isNull(httpConfig.getConnectionTimeout()) ? (int) HttpConstants.CLIENT_POLLING_CONNECT_TIMEOUT : httpConfig.getConnectionTimeout());
+        factory.setReadTimeout(Objects.isNull(httpConfig.getReadTimeout()) ? (int) HttpConstants.CLIENT_POLLING_READ_TIMEOUT : httpConfig.getReadTimeout());
+        factory.setWriteTimeout(Objects.isNull(httpConfig.getWriteTimeout()) ? (int) HttpConstants.CLIENT_POLLING_WRITE_TIMEOUT : httpConfig.getWriteTimeout());
+        return new RestTemplate(factory);
+    }
+
+    /**
+     * AccessTokenManager.
+     *
+     * @param httpConfig   the http config.
+     * @param restTemplate the rest template.
+     * @return the access token manager.
+     */
+    @Bean
+    public AccessTokenManager accessTokenManager(final HttpConfig httpConfig, final RestTemplate restTemplate) {
+        return new AccessTokenManager(restTemplate, httpConfig);
     }
 
     /**
-     * Http config.
+     * Http sync data service.
      *
-     * @return the http config
+     * @param httpConfig         the http config
+     * @param pluginSubscriber   the plugin subscriber
+     * @param restTemplate       the rest template
+     * @param metaSubscribers    the meta subscribers
+     * @param authSubscribers    the auth subscribers
+     * @param accessTokenManager the access token manager
+     * @return the sync data service
      */
     @Bean
-    @ConfigurationProperties(prefix = "shenyu.sync.http")
-    public HttpConfig httpConfig() {
-        return new HttpConfig();
+    public SyncDataService httpSyncDataService(final ObjectProvider<HttpConfig> httpConfig,
+                                               final ObjectProvider<PluginDataSubscriber> pluginSubscriber,
+                                               final ObjectProvider<RestTemplate> restTemplate,
+                                               final ObjectProvider<List<MetaDataSubscriber>> metaSubscribers,
+                                               final ObjectProvider<List<AuthDataSubscriber>> authSubscribers,
+                                               final ObjectProvider<AccessTokenManager> accessTokenManager) {
+        LOGGER.info("you use http long pull sync shenyu data");
+        return new HttpSyncDataService(
+                Objects.requireNonNull(httpConfig.getIfAvailable()),
+                Objects.requireNonNull(pluginSubscriber.getIfAvailable()),
+                Objects.requireNonNull(restTemplate.getIfAvailable()),
+                metaSubscribers.getIfAvailable(Collections::emptyList),
+                authSubscribers.getIfAvailable(Collections::emptyList),
+                Objects.requireNonNull(accessTokenManager.getIfAvailable())
+        );
     }
 }
diff --git a/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/test/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpClientPluginConfigurationTest.java b/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/test/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpClientPluginConfigurationTest.java
index 360265960..819e7be75 100644
--- a/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/test/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpClientPluginConfigurationTest.java
+++ b/shenyu-spring-boot-starter/shenyu-spring-boot-starter-sync-data-center/shenyu-spring-boot-starter-sync-data-http/src/test/java/org/apache/shenyu/springboot/starter/sync/data/http/HttpClientPluginConfigurationTest.java
@@ -46,9 +46,9 @@ import static com.github.tomakehurst.wiremock.client.WireMock.get;
 import static com.github.tomakehurst.wiremock.client.WireMock.post;
 import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo;
 import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.options;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.hamcrest.MatcherAssert.assertThat;
 
 /**
  * Test cases for {@link HttpSyncDataConfiguration}.
@@ -133,6 +133,7 @@ public final class HttpClientPluginConfigurationTest {
         Map<String, Object> result = new HashMap<>();
         Map<String, Object> data = new HashMap<>();
         data.put("token", "token");
+        data.put("expiredTime", 24 * 60 * 60 * 1000);
         result.put("data", data);
         result.put("code", CommonErrorCode.SUCCESSFUL);
         return GsonUtils.getInstance().toJson(result);
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/AccessTokenManager.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/AccessTokenManager.java
new file mode 100644
index 000000000..1e4049730
--- /dev/null
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/AccessTokenManager.java
@@ -0,0 +1,137 @@
+/*
+ * 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.shenyu.sync.data.http;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import org.apache.shenyu.common.concurrent.ShenyuThreadFactory;
+import org.apache.shenyu.common.constant.Constants;
+import org.apache.shenyu.common.exception.CommonErrorCode;
+import org.apache.shenyu.common.utils.GsonUtils;
+import org.apache.shenyu.sync.data.http.config.HttpConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.web.client.RestClientException;
+import org.springframework.web.client.RestTemplate;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * AccessTokenManager.
+ */
+public class AccessTokenManager {
+
+    public static final Logger LOG = LoggerFactory.getLogger(AccessTokenManager.class);
+
+    /**
+     * the access token.
+     */
+    private volatile String accessToken;
+
+    /**
+     * TTL of token in seconds.
+     */
+    private long tokenExpiredTime;
+
+    /**
+     * Last timestamp refresh security info from server.
+     */
+    private long lastRefreshTime;
+
+    /**
+     * Time window to refresh security info in seconds.
+     */
+    private long tokenRefreshWindow;
+
+    private final RestTemplate restTemplate;
+
+    private final HttpConfig httpConfig;
+
+    private final ScheduledExecutorService executorService;
+
+    /**
+     * Construct.
+     *
+     * @param restTemplate the rest template.
+     * @param httpConfig   the config.
+     */
+    public AccessTokenManager(final RestTemplate restTemplate, final HttpConfig httpConfig) {
+        this.restTemplate = restTemplate;
+        this.httpConfig = httpConfig;
+        this.executorService = new ScheduledThreadPoolExecutor(1, ShenyuThreadFactory.create("http-long-polling-client-token-refresh", true));
+        this.start(Lists.newArrayList(Splitter.on(",").split(httpConfig.getUrl())));
+    }
+
+    /**
+     * server login.
+     *
+     * @param servers server list.
+     */
+    public void login(final List<String> servers) {
+        if ((System.currentTimeMillis() - lastRefreshTime) < (tokenExpiredTime - tokenRefreshWindow)) {
+            return;
+        }
+        for (String server : servers) {
+            if (this.doLogin(server)) {
+                this.lastRefreshTime = System.currentTimeMillis();
+                return;
+            }
+        }
+    }
+
+    private Boolean doLogin(final String server) {
+        String param = Constants.LOGIN_NAME + "=" + httpConfig.getUsername() + "&" + Constants.PASS_WORD + "=" + httpConfig.getPassword();
+        String url = String.join("?", server + Constants.LOGIN_PATH, param);
+        try {
+            String result = this.restTemplate.getForObject(url, String.class);
+            Map<String, Object> resultMap = GsonUtils.getInstance().convertToMap(result);
+            if (!String.valueOf(CommonErrorCode.SUCCESSFUL).equals(String.valueOf(resultMap.get(Constants.ADMIN_RESULT_CODE)))) {
+                LOG.warn(String.format("get token from server : [%s] error", server));
+                return false;
+            }
+            String tokenJson = GsonUtils.getInstance().toJson(resultMap.get(Constants.ADMIN_RESULT_DATA));
+            LOG.info("login success: {} ", tokenJson);
+            Map<String, Object> tokenMap = GsonUtils.getInstance().convertToMap(tokenJson);
+            this.accessToken = (String) tokenMap.get(Constants.ADMIN_RESULT_TOKEN);
+            this.tokenExpiredTime = (long) tokenMap.get(Constants.ADMIN_RESULT_EXPIRED_TIME);
+            this.tokenRefreshWindow = this.tokenExpiredTime / 10;
+            return true;
+        } catch (RestClientException e) {
+            LOG.error(String.format("get token from server : [%s] error", server), e);
+            return false;
+        }
+    }
+
+    private void start(final List<String> servers) {
+        this.login(servers);
+        this.executorService.scheduleWithFixedDelay(() -> this.login(servers), 5000, 5000, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * get access token.
+     *
+     * @return the access token
+     */
+    public String getAccessToken() {
+        return accessToken;
+    }
+}
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/HttpSyncDataService.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/HttpSyncDataService.java
index 1c9a557c1..237e9a207 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/HttpSyncDataService.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/HttpSyncDataService.java
@@ -21,16 +21,12 @@ import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 import com.google.gson.JsonArray;
 import com.google.gson.JsonObject;
-import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.shenyu.common.concurrent.ShenyuThreadFactory;
 import org.apache.shenyu.common.constant.Constants;
-import org.apache.shenyu.common.constant.HttpConstants;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
-import org.apache.shenyu.common.exception.CommonErrorCode;
 import org.apache.shenyu.common.exception.ShenyuException;
-import org.apache.shenyu.common.utils.FreshBeanHolder;
 import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.common.utils.ThreadUtils;
 import org.apache.shenyu.sync.data.api.AuthDataSubscriber;
@@ -45,7 +41,6 @@ import org.springframework.http.HttpEntity;
 import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpMethod;
 import org.springframework.http.MediaType;
-import org.springframework.http.client.OkHttp3ClientHttpRequestFactory;
 import org.springframework.util.LinkedMultiValueMap;
 import org.springframework.util.MultiValueMap;
 import org.springframework.web.client.RestClientException;
@@ -53,9 +48,7 @@ import org.springframework.web.client.RestTemplate;
 
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -77,36 +70,29 @@ public class HttpSyncDataService implements SyncDataService {
     /**
      * only use for http long polling.
      */
-    private final RestTemplate httpClient;
+    private final RestTemplate restTemplate;
 
     private ExecutorService executor;
 
-    private final HttpConfig httpConfig;
-
     private final List<String> serverList;
 
     private final DataRefreshFactory factory;
 
-    private final FreshBeanHolder<String, Optional<Object>> accessToken;
+    private final AccessTokenManager accessTokenManager;
 
-    public HttpSyncDataService(final HttpConfig httpConfig, final PluginDataSubscriber pluginDataSubscriber,
-                               final List<MetaDataSubscriber> metaDataSubscribers, final List<AuthDataSubscriber> authDataSubscribers) {
-        this.httpConfig = httpConfig;
+    public HttpSyncDataService(final HttpConfig httpConfig,
+                               final PluginDataSubscriber pluginDataSubscriber,
+                               final RestTemplate restTemplate,
+                               final List<MetaDataSubscriber> metaDataSubscribers,
+                               final List<AuthDataSubscriber> authDataSubscribers,
+                               final AccessTokenManager accessTokenManager) {
+        this.accessTokenManager = accessTokenManager;
         this.factory = new DataRefreshFactory(pluginDataSubscriber, metaDataSubscribers, authDataSubscribers);
         this.serverList = Lists.newArrayList(Splitter.on(",").split(httpConfig.getUrl()));
-        this.httpClient = createRestTemplate(httpConfig);
-        this.accessToken = new FreshBeanHolder<>(this::doLogin);
+        this.restTemplate = restTemplate;
         this.start();
     }
 
-    private RestTemplate createRestTemplate(final HttpConfig httpConfig) {
-        OkHttp3ClientHttpRequestFactory factory = new OkHttp3ClientHttpRequestFactory();
-        factory.setConnectTimeout(Objects.isNull(httpConfig.getConnectionTimeout()) ? (int) HttpConstants.CLIENT_POLLING_CONNECT_TIMEOUT : httpConfig.getConnectionTimeout());
-        factory.setReadTimeout(Objects.isNull(httpConfig.getReadTimeout()) ? (int) HttpConstants.CLIENT_POLLING_READ_TIMEOUT : httpConfig.getReadTimeout());
-        factory.setWriteTimeout(Objects.isNull(httpConfig.getWriteTimeout()) ? (int) HttpConstants.CLIENT_POLLING_WRITE_TIMEOUT : httpConfig.getWriteTimeout());
-        return new RestTemplate(factory);
-    }
-
     private void start() {
         // It could be initialized multiple times, so you need to control that.
         if (RUNNING.compareAndSet(false, true)) {
@@ -148,14 +134,10 @@ public class HttpSyncDataService implements SyncDataService {
         LOG.info("request configs: [{}]", url);
         String json;
         try {
-            Optional<Object> token = accessToken.apply(server);
-            if (!token.isPresent()) {
-                throw new ShenyuException("get token from server : [" + server + " ] error");
-            }
             HttpHeaders headers = new HttpHeaders();
-            headers.set(Constants.X_ACCESS_TOKEN, String.valueOf(token.get()));
+            headers.set(Constants.X_ACCESS_TOKEN, this.accessTokenManager.getAccessToken());
             HttpEntity<String> httpEntity = new HttpEntity<>(headers);
-            json = this.httpClient.exchange(url, HttpMethod.GET, httpEntity, String.class).getBody();
+            json = this.restTemplate.exchange(url, HttpMethod.GET, httpEntity, String.class).getBody();
         } catch (RestClientException e) {
             String message = String.format("fetch config fail from server[%s], %s", url, e.getMessage());
             LOG.warn(message);
@@ -164,7 +146,7 @@ public class HttpSyncDataService implements SyncDataService {
         // update local cache
         boolean updated = this.updateCacheWithJson(json);
         if (updated) {
-            LOG.info("get latest configs: [{}]", json);
+            LOG.debug("get latest configs: [{}]", json);
             return;
         }
         // not updated. it is likely that the current config server has not been updated yet. wait a moment.
@@ -172,6 +154,8 @@ public class HttpSyncDataService implements SyncDataService {
         ThreadUtils.sleep(TimeUnit.SECONDS, 30);
     }
 
+
+
     /**
      * update local cache.
      *
@@ -180,16 +164,11 @@ public class HttpSyncDataService implements SyncDataService {
      */
     private boolean updateCacheWithJson(final String json) {
         JsonObject jsonObject = GsonUtils.getGson().fromJson(json, JsonObject.class);
-        JsonObject data = jsonObject.getAsJsonObject("data");
         // if the config cache will be updated?
-        return factory.executor(data);
+        return factory.executor(jsonObject.getAsJsonObject("data"));
     }
 
     private void doLongPolling(final String server) {
-        Optional<Object> token = accessToken.apply(server);
-        if (!token.isPresent()) {
-            throw new ShenyuException("get token from server : [" + server + " ] error");
-        }
         MultiValueMap<String, String> params = new LinkedMultiValueMap<>(8);
         for (ConfigGroupEnum group : ConfigGroupEnum.values()) {
             ConfigData<?> cacheConfig = factory.cacheConfigData(group);
@@ -200,28 +179,26 @@ public class HttpSyncDataService implements SyncDataService {
         }
         HttpHeaders headers = new HttpHeaders();
         headers.setContentType(MediaType.APPLICATION_FORM_URLENCODED);
-        headers.set(Constants.X_ACCESS_TOKEN, String.valueOf(token.get()));
+        headers.set(Constants.X_ACCESS_TOKEN, this.accessTokenManager.getAccessToken());
         HttpEntity<MultiValueMap<String, String>> httpEntity = new HttpEntity<>(params, headers);
         String listenerUrl = server + Constants.SHENYU_ADMIN_PATH_CONFIGS_LISTENER;
-        LOG.debug("request listener configs: [{}]", listenerUrl);
 
         JsonArray groupJson;
         try {
-            String json = this.httpClient.postForEntity(listenerUrl, httpEntity, String.class).getBody();
-            LOG.debug("listener result: [{}]", json);
-            groupJson = GsonUtils.getGson().fromJson(json, JsonObject.class).getAsJsonArray("data");
+            String json = this.restTemplate.postForEntity(listenerUrl, httpEntity, String.class).getBody();
+            LOG.info("listener result: [{}]", json);
+            JsonObject responseFromServer = GsonUtils.getGson().fromJson(json, JsonObject.class);
+            groupJson = responseFromServer.getAsJsonArray("data");
         } catch (RestClientException e) {
             String message = String.format("listener configs fail, server:[%s], %s", server, e.getMessage());
             throw new ShenyuException(message, e);
         }
 
-        if (Objects.nonNull(groupJson)) {
+        if (Objects.nonNull(groupJson) && groupJson.size() > 0) {
             // fetch group configuration async.
             ConfigGroupEnum[] changedGroups = GsonUtils.getGson().fromJson(groupJson, ConfigGroupEnum[].class);
-            if (ArrayUtils.isNotEmpty(changedGroups)) {
-                LOG.info("Group config changed: {}", Arrays.toString(changedGroups));
-                this.doFetchGroupConfig(server, changedGroups);
-            }
+            LOG.info("Group config changed: {}", Arrays.toString(changedGroups));
+            this.doFetchGroupConfig(server, changedGroups);
         }
     }
 
@@ -235,20 +212,6 @@ public class HttpSyncDataService implements SyncDataService {
         }
     }
 
-    private Optional<Object> doLogin(final String server) {
-        String param = Constants.LOGIN_NAME + "=" + httpConfig.getUsername() + "&" + Constants.PASS_WORD + "=" + httpConfig.getPassword();
-        String url = String.join("?", server + Constants.LOGIN_PATH, param);
-        String result = this.httpClient.getForObject(url, String.class);
-        Map<String, Object> resultMap = GsonUtils.getInstance().convertToMap(result);
-        if (!String.valueOf(CommonErrorCode.SUCCESSFUL).equals(String.valueOf(resultMap.get(Constants.ADMIN_RESULT_CODE)))) {
-            return Optional.empty();
-        }
-        String tokenJson = GsonUtils.getInstance().toJson(resultMap.get(Constants.ADMIN_RESULT_DATA));
-        LOG.info("login success: {} ", tokenJson);
-        Map<String, Object> tokenMap = GsonUtils.getInstance().convertToMap(tokenJson);
-        return Optional.ofNullable(tokenMap.get(Constants.ADMIN_RESULT_TOKEN));
-    }
-
     class HttpLongPollingTask implements Runnable {
 
         private final String server;
@@ -263,8 +226,7 @@ public class HttpSyncDataService implements SyncDataService {
                 int retryTimes = 3;
                 for (int time = 1; time <= retryTimes; time++) {
                     try {
-                        //refresh the admin token
-                        accessToken.doFresh(server);
+                        //do long polling.
                         doLongPolling(server);
                     } catch (Exception e) {
                         // print warnning LOG.
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/config/HttpConfig.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/config/HttpConfig.java
index d0a410ced..c9d99bc21 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/config/HttpConfig.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/config/HttpConfig.java
@@ -17,11 +17,14 @@
 
 package org.apache.shenyu.sync.data.http.config;
 
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
 import java.util.Objects;
 
 /**
  * The type Http config.
  */
+@ConfigurationProperties(prefix = "shenyu.sync.http")
 public class HttpConfig {
 
     private String url;
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AbstractDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AbstractDataRefresh.java
index 6ba66dc74..2721de66f 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AbstractDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AbstractDataRefresh.java
@@ -17,7 +17,6 @@
 
 package org.apache.shenyu.sync.data.http.refresh;
 
-import com.google.gson.Gson;
 import com.google.gson.JsonObject;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.shenyu.common.dto.ConfigData;
@@ -42,11 +41,6 @@ public abstract class AbstractDataRefresh<T> implements DataRefresh {
      */
     protected static final ConcurrentMap<ConfigGroupEnum, ConfigData<?>> GROUP_CACHE = new ConcurrentHashMap<>();
 
-    /**
-     * The constant GSON.
-     */
-    protected static final Gson GSON = new Gson();
-
     /**
      * logger.
      */
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AppAuthDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AppAuthDataRefresh.java
index 66a4ea617..944a96b7d 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AppAuthDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/AppAuthDataRefresh.java
@@ -23,6 +23,7 @@ import org.apache.commons.collections4.CollectionUtils;
 import org.apache.shenyu.common.dto.AppAuthData;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
+import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.sync.data.api.AuthDataSubscriber;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +53,7 @@ public class AppAuthDataRefresh extends AbstractDataRefresh<AppAuthData> {
 
     @Override
     protected ConfigData<AppAuthData> fromJson(final JsonObject data) {
-        return GSON.fromJson(data, new TypeToken<ConfigData<AppAuthData>>() {
+        return GsonUtils.getGson().fromJson(data, new TypeToken<ConfigData<AppAuthData>>() {
         }.getType());
     }
 
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/MetaDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/MetaDataRefresh.java
index e1efddf03..e249ea28f 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/MetaDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/MetaDataRefresh.java
@@ -19,15 +19,17 @@ package org.apache.shenyu.sync.data.http.refresh;
 
 import com.google.gson.JsonObject;
 import com.google.gson.reflect.TypeToken;
-import java.util.List;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.dto.MetaData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
+import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.sync.data.api.MetaDataSubscriber;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.List;
+
 /**
  * The type meta data refresh.
  */
@@ -50,7 +52,7 @@ public class MetaDataRefresh extends AbstractDataRefresh<MetaData> {
 
     @Override
     protected ConfigData<MetaData> fromJson(final JsonObject data) {
-        return GSON.fromJson(data, new TypeToken<ConfigData<MetaData>>() {
+        return GsonUtils.getGson().fromJson(data, new TypeToken<ConfigData<MetaData>>() {
         }.getType());
     }
 
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/PluginDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/PluginDataRefresh.java
index c5b4716aa..f9027d7f1 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/PluginDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/PluginDataRefresh.java
@@ -23,6 +23,7 @@ import org.apache.commons.collections4.CollectionUtils;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.dto.PluginData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
+import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.sync.data.api.PluginDataSubscriber;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +53,7 @@ public class PluginDataRefresh extends AbstractDataRefresh<PluginData> {
 
     @Override
     protected ConfigData<PluginData> fromJson(final JsonObject data) {
-        return GSON.fromJson(data, new TypeToken<ConfigData<PluginData>>() {
+        return GsonUtils.getGson().fromJson(data, new TypeToken<ConfigData<PluginData>>() {
         }.getType());
     }
 
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/RuleDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/RuleDataRefresh.java
index 01947b1b9..08b7542d7 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/RuleDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/RuleDataRefresh.java
@@ -19,15 +19,17 @@ package org.apache.shenyu.sync.data.http.refresh;
 
 import com.google.gson.JsonObject;
 import com.google.gson.reflect.TypeToken;
-import java.util.List;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.dto.RuleData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
+import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.sync.data.api.PluginDataSubscriber;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.List;
+
 /**
  * The type Rule data refresh.
  */
@@ -51,7 +53,7 @@ public class RuleDataRefresh extends AbstractDataRefresh<RuleData> {
 
     @Override
     protected ConfigData<RuleData> fromJson(final JsonObject data) {
-        return GSON.fromJson(data, new TypeToken<ConfigData<RuleData>>() {
+        return GsonUtils.getGson().fromJson(data, new TypeToken<ConfigData<RuleData>>() {
         }.getType());
     }
 
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/SelectorDataRefresh.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/SelectorDataRefresh.java
index 96962501b..7d6fecd93 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/SelectorDataRefresh.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/main/java/org/apache/shenyu/sync/data/http/refresh/SelectorDataRefresh.java
@@ -23,6 +23,7 @@ import org.apache.commons.collections4.CollectionUtils;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.dto.SelectorData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
+import org.apache.shenyu.common.utils.GsonUtils;
 import org.apache.shenyu.sync.data.api.PluginDataSubscriber;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +53,7 @@ public class SelectorDataRefresh extends AbstractDataRefresh<SelectorData> {
 
     @Override
     protected ConfigData<SelectorData> fromJson(final JsonObject data) {
-        return GSON.fromJson(data, new TypeToken<ConfigData<SelectorData>>() {
+        return GsonUtils.getGson().fromJson(data, new TypeToken<ConfigData<SelectorData>>() {
         }.getType());
     }
 
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/AccessTokenManagerTest.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/AccessTokenManagerTest.java
new file mode 100644
index 000000000..6a89e2a3a
--- /dev/null
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/AccessTokenManagerTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.shenyu.sync.data.http;
+
+import com.github.tomakehurst.wiremock.WireMockServer;
+import com.github.tomakehurst.wiremock.extension.responsetemplating.ResponseTemplateTransformer;
+import com.google.common.collect.Lists;
+import org.apache.shenyu.common.constant.HttpConstants;
+import org.apache.shenyu.common.exception.CommonErrorCode;
+import org.apache.shenyu.common.utils.GsonUtils;
+import org.apache.shenyu.sync.data.http.config.HttpConfig;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.http.client.OkHttp3ClientHttpRequestFactory;
+import org.springframework.web.client.RestTemplate;
+import wiremock.org.apache.http.HttpHeaders;
+import wiremock.org.apache.http.entity.ContentType;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
+import static com.github.tomakehurst.wiremock.client.WireMock.get;
+import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo;
+import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.options;
+
+/**
+ * AccessTokenManagerTest.
+ */
+@ExtendWith(MockitoExtension.class)
+public class AccessTokenManagerTest {
+
+    /**
+     * logger.
+     */
+    private static final Logger LOG = LoggerFactory.getLogger(AccessTokenManagerTest.class);
+
+    private WireMockServer wireMockServer;
+
+    private AccessTokenManager accessTokenManager;
+
+    private HttpConfig httpConfig;
+
+    private String accessToken;
+
+    @BeforeEach
+    public void before() {
+        this.wireMockServer = new WireMockServer(
+                options()
+                        .extensions(new ResponseTemplateTransformer(false))
+                        .dynamicPort());
+        this.wireMockServer.start();
+        wireMockServer.stubFor(get(urlPathEqualTo("/platform/login"))
+                .willReturn(aResponse()
+                        .withHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.toString())
+                        .withBody(this.mockLoginResponseJson())
+                        .withStatus(200))
+        );
+
+        this.httpConfig = new HttpConfig();
+        httpConfig.setUrl(this.getMockServerUrl());
+        // set http connection timeout
+        httpConfig.setConnectionTimeout(3000);
+        // set delay time
+        httpConfig.setDelayTime(3);
+        httpConfig.setPassword("123456");
+        httpConfig.setUsername("admin");
+
+        OkHttp3ClientHttpRequestFactory factory = new OkHttp3ClientHttpRequestFactory();
+        factory.setConnectTimeout(Objects.isNull(httpConfig.getConnectionTimeout()) ? (int) HttpConstants.CLIENT_POLLING_CONNECT_TIMEOUT : httpConfig.getConnectionTimeout());
+        factory.setReadTimeout(Objects.isNull(httpConfig.getReadTimeout()) ? (int) HttpConstants.CLIENT_POLLING_READ_TIMEOUT : httpConfig.getReadTimeout());
+        factory.setWriteTimeout(Objects.isNull(httpConfig.getWriteTimeout()) ? (int) HttpConstants.CLIENT_POLLING_WRITE_TIMEOUT : httpConfig.getWriteTimeout());
+        RestTemplate restTemplate = new RestTemplate(factory);
+
+        this.accessTokenManager = new AccessTokenManager(restTemplate, httpConfig);
+    }
+
+    @Test
+    public void testLogin() {
+        accessTokenManager.login(Lists.newArrayList(httpConfig.getUrl().split(",")));
+        Assert.assertEquals(this.accessToken, accessTokenManager.getAccessToken());
+    }
+
+    // mock configs fetch api response
+    private String mockLoginResponseJson() {
+        Map<String, Object> data = new HashMap<>();
+        data.put("token", "token");
+        this.accessToken = "token";
+        data.put("expiredTime", 24 * 60 * 60 * 1000);
+        Map<String, Object> result = new HashMap<>();
+        result.put("data", data);
+        result.put("code", CommonErrorCode.SUCCESSFUL);
+        return GsonUtils.getInstance().toJson(result);
+    }
+
+    private String getMockServerUrl() {
+        return "http://127.0.0.1:" + wireMockServer.port();
+    }
+}
diff --git a/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/HttpSyncDataServiceTest.java b/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/HttpSyncDataServiceTest.java
index 2848aa6a2..f5ee4cae4 100644
--- a/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/HttpSyncDataServiceTest.java
+++ b/shenyu-sync-data-center/shenyu-sync-data-http/src/test/java/org/apache/shenyu/sync/data/http/HttpSyncDataServiceTest.java
@@ -19,6 +19,7 @@ package org.apache.shenyu.sync.data.http;
 
 import com.github.tomakehurst.wiremock.WireMockServer;
 import com.github.tomakehurst.wiremock.extension.responsetemplating.ResponseTemplateTransformer;
+import org.apache.shenyu.common.constant.HttpConstants;
 import org.apache.shenyu.common.dto.ConfigData;
 import org.apache.shenyu.common.dto.PluginData;
 import org.apache.shenyu.common.enums.ConfigGroupEnum;
@@ -35,20 +36,22 @@ import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.junit.jupiter.MockitoExtension;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.http.client.OkHttp3ClientHttpRequestFactory;
 import org.springframework.test.util.ReflectionTestUtils;
+import org.springframework.web.client.RestTemplate;
 import wiremock.org.apache.http.HttpHeaders;
 import wiremock.org.apache.http.entity.ContentType;
 
 import java.util.Collections;
-import java.util.Objects;
-import java.util.Map;
 import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
 import static com.github.tomakehurst.wiremock.client.WireMock.get;
 import static com.github.tomakehurst.wiremock.client.WireMock.post;
 import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo;
-import static com.github.tomakehurst.wiremock.client.WireMock.aResponse;
 import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.options;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -111,8 +114,16 @@ public final class HttpSyncDataServiceTest {
         this.pluginDataSubscriber = mock(PluginDataSubscriber.class);
         this.metaDataSubscriber = mock(MetaDataSubscriber.class);
         this.authDataSubscriber = mock(AuthDataSubscriber.class);
-        this.httpSyncDataService = new HttpSyncDataService(httpConfig, pluginDataSubscriber,
-                Collections.singletonList(metaDataSubscriber), Collections.singletonList(authDataSubscriber));
+
+        OkHttp3ClientHttpRequestFactory factory = new OkHttp3ClientHttpRequestFactory();
+        factory.setConnectTimeout(Objects.isNull(httpConfig.getConnectionTimeout()) ? (int) HttpConstants.CLIENT_POLLING_CONNECT_TIMEOUT : httpConfig.getConnectionTimeout());
+        factory.setReadTimeout(Objects.isNull(httpConfig.getReadTimeout()) ? (int) HttpConstants.CLIENT_POLLING_READ_TIMEOUT : httpConfig.getReadTimeout());
+        factory.setWriteTimeout(Objects.isNull(httpConfig.getWriteTimeout()) ? (int) HttpConstants.CLIENT_POLLING_WRITE_TIMEOUT : httpConfig.getWriteTimeout());
+        RestTemplate restTemplate = new RestTemplate(factory);
+
+        AccessTokenManager accessTokenManager = new AccessTokenManager(restTemplate, httpConfig);
+        this.httpSyncDataService = new HttpSyncDataService(httpConfig, pluginDataSubscriber, restTemplate,
+                Collections.singletonList(metaDataSubscriber), Collections.singletonList(authDataSubscriber), accessTokenManager);
     }
 
     @AfterEach
@@ -176,6 +187,7 @@ public final class HttpSyncDataServiceTest {
         Map<String, Object> result = new HashMap<>();
         Map<String, Object> data = new HashMap<>();
         data.put("token", "token");
+        data.put("expiredTime", 24 * 60 * 60 * 1000);
         result.put("data", data);
         result.put("code", CommonErrorCode.SUCCESSFUL);
         return GsonUtils.getInstance().toJson(result);