You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2022/11/10 13:24:33 UTC

[shardingsphere] branch master updated: Refactor return value of WorkerIdGenerator.generate from long to int (#22064)

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

zhaojinchao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new c610bcf281e Refactor return value of WorkerIdGenerator.generate from long to int (#22064)
c610bcf281e is described below

commit c610bcf281ea032166197286e1a36010fa9343c1
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Thu Nov 10 21:24:23 2022 +0800

    Refactor return value of WorkerIdGenerator.generate from long to int (#22064)
---
 .../keygen/SnowflakeKeyGenerateAlgorithm.java      |  6 +--
 .../keygen/SnowflakeKeyGenerateAlgorithmTest.java  |  6 +--
 .../keygen/fixture/WorkerIdGeneratorFixture.java   |  4 +-
 .../keygen/CosIdSnowflakeKeyGenerateAlgorithm.java |  2 +-
 .../CosIdSnowflakeKeyGenerateAlgorithmTest.java    |  8 ++--
 .../keygen/fixture/WorkerIdGeneratorFixture.java   |  4 +-
 .../infra/instance/ComputeNodeInstance.java        | 11 +++---
 .../infra/instance/InstanceContext.java            |  8 ++--
 .../infra/instance/workerid/WorkerIdGenerator.java |  6 +--
 .../infra/instance/InstanceContextTest.java        | 44 +++++++++-------------
 .../instance/fixture/WorkerIdGeneratorFixture.java |  4 +-
 .../builder/global/GlobalRulesBuilderTest.java     |  2 +-
 .../status/compute/event/WorkerIdEvent.java        |  2 +-
 .../compute/service/ComputeNodeStatusService.java  | 16 ++++----
 .../watcher/ComputeNodeStateChangedWatcher.java    |  2 +-
 .../generator/ClusterWorkerIdGenerator.java        | 32 ++++++++--------
 .../service/ComputeNodeStatusServiceTest.java      |  4 +-
 .../generator/ClusterWorkerIdGeneratorTest.java    | 10 ++---
 .../subscriber/StateChangedSubscriberTest.java     |  4 +-
 .../listener/SessionConnectionListener.java        |  2 +-
 .../generator/StandaloneWorkerIdGenerator.java     |  4 +-
 .../generator/StandaloneWorkerIdGeneratorTest.java |  2 +-
 .../queryable/ShowComputeNodeInfoHandlerTest.java  |  4 +-
 .../ral/queryable/ShowComputeNodesHandlerTest.java |  8 ++--
 .../parser/result/SQLParserLogResultProcessor.java |  2 +-
 25 files changed, 92 insertions(+), 105 deletions(-)

diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
index 12e2f250204..bcae18bb199 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithm.java
@@ -61,7 +61,7 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
     
     private static final int MAX_TOLERATE_TIME_DIFFERENCE_MILLISECONDS = 10;
     
-    private static final long DEFAULT_WORKER_ID = 0;
+    private static final int DEFAULT_WORKER_ID = 0;
     
     @Setter
     private static TimeService timeService = new TimeService();
@@ -131,7 +131,7 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
             sequence = sequenceOffset;
         }
         lastMilliseconds = currentMilliseconds;
-        return ((currentMilliseconds - EPOCH) << TIMESTAMP_LEFT_SHIFT_BITS) | (getWorkerId() << WORKER_ID_LEFT_SHIFT_BITS) | sequence;
+        return ((currentMilliseconds - EPOCH) << TIMESTAMP_LEFT_SHIFT_BITS) | ((long) getWorkerId() << WORKER_ID_LEFT_SHIFT_BITS) | sequence;
     }
     
     @SneakyThrows(InterruptedException.class)
@@ -159,7 +159,7 @@ public final class SnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgorithm
         sequenceOffset = sequenceOffset >= maxVibrationOffset ? 0 : sequenceOffset + 1;
     }
     
-    private long getWorkerId() {
+    private int getWorkerId() {
         return null == instanceContext ? DEFAULT_WORKER_ID : instanceContext.getWorkerId();
     }
     
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
index bf01cc8004d..db2b501bbba 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
@@ -60,7 +60,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     
     static {
         InstanceContext instanceContext = mock(InstanceContext.class);
-        when(instanceContext.getWorkerId()).thenReturn(0L);
+        when(instanceContext.getWorkerId()).thenReturn(0);
         INSTANCE = instanceContext;
     }
     
@@ -206,7 +206,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     @Test(expected = IllegalArgumentException.class)
     public void assertSetWorkerIdFailureWhenNegative() {
         SnowflakeKeyGenerateAlgorithm algorithm = (SnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("SNOWFLAKE", new Properties()));
-        InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(-1L),
+        InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(-1),
                 new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext());
         algorithm.setInstanceContext(instanceContext);
         algorithm.generateKey();
@@ -222,7 +222,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     @Test(expected = IllegalArgumentException.class)
     public void assertSetWorkerIdFailureWhenOutOfRange() {
         SnowflakeKeyGenerateAlgorithm algorithm = (SnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("SNOWFLAKE", new Properties()));
-        InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
+        InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE),
                 new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext());
         algorithm.setInstanceContext(instanceContext);
         algorithm.generateKey();
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
index 3115fb49f81..10218a80bb4 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
@@ -26,10 +26,10 @@ import java.util.Properties;
 @RequiredArgsConstructor
 public final class WorkerIdGeneratorFixture implements WorkerIdGenerator {
     
-    private final long fixtureWorkerId;
+    private final int fixtureWorkerId;
     
     @Override
-    public long generate(final Properties props) {
+    public int generate(final Properties props) {
         Preconditions.checkArgument(fixtureWorkerId >= 0L && fixtureWorkerId <= MAX_WORKER_ID, "Illegal worker id.");
         return fixtureWorkerId;
     }
diff --git a/features/sharding/plugin/cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithm.java b/features/sharding/plugin/cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithm.java
index 5aff3810e35..75768dc2c11 100644
--- a/features/sharding/plugin/cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithm.java
+++ b/features/sharding/plugin/cosid/src/main/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithm.java
@@ -79,7 +79,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithm implements KeyGenerateAlgo
     
     @Override
     public void setInstanceContext(final InstanceContext instanceContext) {
-        long workerId = instanceContext.generateWorkerId(props);
+        int workerId = instanceContext.generateWorkerId(props);
         MillisecondSnowflakeId millisecondSnowflakeId =
                 new MillisecondSnowflakeId(epoch, MillisecondSnowflakeId.DEFAULT_TIMESTAMP_BIT, MillisecondSnowflakeId.DEFAULT_MACHINE_BIT, MillisecondSnowflakeId.DEFAULT_SEQUENCE_BIT, workerId);
         snowflakeId = new StringSnowflakeId(new ClockSyncSnowflakeId(millisecondSnowflakeId), Radix62IdConverter.PAD_START);
diff --git a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
index 6028213c7ec..1a67887f2d8 100644
--- a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
+++ b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
@@ -45,7 +45,7 @@ import static org.mockito.Mockito.mock;
 
 public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
     
-    private static final long FIXTURE_WORKER_ID = 0;
+    private static final int FIXTURE_WORKER_ID = 0;
     
     private final SnowflakeIdStateParser snowflakeIdStateParser = new MillisecondSnowflakeIdStateParser(
             CosIdSnowflakeKeyGenerateAlgorithm.DEFAULT_EPOCH,
@@ -65,9 +65,9 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         long secondActualKey = (Long) algorithm.generateKey();
         SnowflakeIdState firstActualState = snowflakeIdStateParser.parse(firstActualKey);
         SnowflakeIdState secondActualState = snowflakeIdStateParser.parse(secondActualKey);
-        assertThat(firstActualState.getMachineId(), is(FIXTURE_WORKER_ID));
+        assertThat(firstActualState.getMachineId(), is((long) FIXTURE_WORKER_ID));
         assertThat(firstActualState.getSequence(), is(1L));
-        assertThat(secondActualState.getMachineId(), is(FIXTURE_WORKER_ID));
+        assertThat(secondActualState.getMachineId(), is((long) FIXTURE_WORKER_ID));
         long expectedSecondSequence = 2L;
         assertThat(secondActualState.getSequence(), is(expectedSecondSequence));
     }
@@ -128,7 +128,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         assertThat(actualStringKey.length(), is(Radix62IdConverter.MAX_CHAR_SIZE));
         long actualLongKey = Radix62IdConverter.PAD_START.asLong(actualStringKey);
         SnowflakeIdState actualState = snowflakeIdStateParser.parse(actualLongKey);
-        assertThat(actualState.getMachineId(), is(FIXTURE_WORKER_ID));
+        assertThat(actualState.getMachineId(), is((long) FIXTURE_WORKER_ID));
         assertThat(actualState.getSequence(), is(1L));
     }
     
diff --git a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
index 3673013cfd5..1587f46ef25 100644
--- a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
+++ b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/fixture/WorkerIdGeneratorFixture.java
@@ -25,10 +25,10 @@ import java.util.Properties;
 @RequiredArgsConstructor
 public final class WorkerIdGeneratorFixture implements WorkerIdGenerator {
     
-    private final long fixtureWorkerId;
+    private final int fixtureWorkerId;
     
     @Override
-    public long generate(final Properties props) {
+    public int generate(final Properties props) {
         return fixtureWorkerId;
     }
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/ComputeNodeInstance.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/ComputeNodeInstance.java
index c0293b7837b..206ffed2ad8 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/ComputeNodeInstance.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/ComputeNodeInstance.java
@@ -30,7 +30,6 @@ import java.util.Collection;
  * Instance of compute node.
  */
 @Getter
-@Setter
 public final class ComputeNodeInstance {
     
     private final InstanceMetaData metaData;
@@ -39,11 +38,12 @@ public final class ComputeNodeInstance {
     
     private Collection<String> labels = new ArrayList<>();
     
-    private volatile long workerId;
+    @Setter
+    private volatile int workerId;
     
     public ComputeNodeInstance(final InstanceMetaData metaData) {
         this.metaData = metaData;
-        workerId = -1L;
+        workerId = -1;
     }
     
     /**
@@ -52,10 +52,9 @@ public final class ComputeNodeInstance {
      * @param labels labels
      */
     public void setLabels(final Collection<String> labels) {
-        if (null == labels) {
-            return;
+        if (null != labels) {
+            this.labels = labels;
         }
-        this.labels = labels;
     }
     
     /**
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
index 3eb59664b1e..6a911b8227b 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
@@ -81,7 +81,7 @@ public final class InstanceContext {
      * @param instanceId instance id
      * @param workerId worker id
      */
-    public void updateWorkerId(final String instanceId, final Long workerId) {
+    public void updateWorkerId(final String instanceId, final Integer workerId) {
         if (instance.getMetaData().getId().equals(instanceId)) {
             instance.setWorkerId(workerId);
         }
@@ -106,7 +106,7 @@ public final class InstanceContext {
      *
      * @return worker id
      */
-    public long getWorkerId() {
+    public int getWorkerId() {
         return instance.getWorkerId();
     }
     
@@ -116,8 +116,8 @@ public final class InstanceContext {
      * @param props properties
      * @return worker id
      */
-    public long generateWorkerId(final Properties props) {
-        long result = workerIdGenerator.generate(props);
+    public int generateWorkerId(final Properties props) {
+        int result = workerIdGenerator.generate(props);
         instance.setWorkerId(result);
         return result;
     }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/workerid/WorkerIdGenerator.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/workerid/WorkerIdGenerator.java
index 5baa182d4af..f0f9b676e29 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/workerid/WorkerIdGenerator.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/workerid/WorkerIdGenerator.java
@@ -26,9 +26,9 @@ public interface WorkerIdGenerator {
     
     String WORKER_ID_KEY = "worker-id";
     
-    long DEFAULT_WORKER_ID = 0L;
+    int DEFAULT_WORKER_ID = 0;
     
-    long MAX_WORKER_ID = 1023L;
+    int MAX_WORKER_ID = 1023;
     
     /**
      * Generate worker id.
@@ -36,5 +36,5 @@ public interface WorkerIdGenerator {
      * @param props props
      * @return worker id
      */
-    long generate(Properties props);
+    int generate(Properties props);
 }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
index 04a36e310ad..fba18d1c02d 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
@@ -18,12 +18,11 @@
 package org.apache.shardingsphere.infra.instance;
 
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
-import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
-import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.instance.fixture.WorkerIdGeneratorFixture;
+import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.lock.LockContext;
-import org.apache.shardingsphere.infra.state.StateContext;
 import org.apache.shardingsphere.infra.state.StateType;
+import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -34,9 +33,9 @@ import java.util.Properties;
 import java.util.Set;
 
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -53,8 +52,7 @@ public final class InstanceContextTest {
     public void assertUpdateInstanceStatus() {
         InstanceMetaData instanceMetaData = mock(InstanceMetaData.class);
         when(instanceMetaData.getId()).thenReturn("foo_instance_id");
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext);
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig, lockContext, eventBusContext);
         StateType actual = context.getInstance().getState().getCurrentState();
         assertThat(actual, is(StateType.OK));
         context.updateInstanceStatus(instanceMetaData.getId(), Collections.singleton(StateType.CIRCUIT_BREAK.name()));
@@ -68,26 +66,23 @@ public final class InstanceContextTest {
     @Test
     public void assertGetWorkerId() {
         ComputeNodeInstance computeNodeInstance = mock(ComputeNodeInstance.class);
-        when(computeNodeInstance.getWorkerId()).thenReturn(0L);
-        InstanceContext context = new InstanceContext(computeNodeInstance, new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext);
-        assertThat(context.getWorkerId(), is(0L));
+        when(computeNodeInstance.getWorkerId()).thenReturn(0);
+        InstanceContext context = new InstanceContext(computeNodeInstance, new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig, lockContext, eventBusContext);
+        assertThat(context.getWorkerId(), is(0));
     }
     
     @Test
     public void assertGenerateWorkerId() {
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext);
-        long actual = context.generateWorkerId(new Properties());
-        assertThat(actual, is(Long.MIN_VALUE));
+        InstanceContext context = new InstanceContext(
+                new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig, lockContext, eventBusContext);
+        assertThat(context.generateWorkerId(new Properties()), is(Integer.MIN_VALUE));
     }
     
     @Test
     public void assertUpdateLabel() {
         InstanceMetaData instanceMetaData = mock(InstanceMetaData.class);
         when(instanceMetaData.getId()).thenReturn("foo_instance_id");
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext);
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig, lockContext, eventBusContext);
         Set<String> expected = new LinkedHashSet<>(Arrays.asList("label_1", "label_2"));
         context.updateLabel("foo_instance_id", expected);
         Collection<String> actual = context.getInstance().getLabels();
@@ -97,34 +92,31 @@ public final class InstanceContextTest {
     @Test
     public void assertGetInstance() {
         ComputeNodeInstance expected = new ComputeNodeInstance(mock(InstanceMetaData.class));
-        InstanceContext context = new InstanceContext(expected, new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext);
+        InstanceContext context = new InstanceContext(expected, new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig, lockContext, eventBusContext);
         ComputeNodeInstance actual = context.getInstance();
         assertThat(actual, is(expected));
     }
     
     @Test
     public void assertGetState() {
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig,
                 lockContext, eventBusContext);
-        StateContext actual = context.getInstance().getState();
-        assertNotNull(actual);
+        assertNotNull(context.getInstance().getState());
     }
     
     @Test
     public void assertGetModeConfiguration() {
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig,
                 lockContext, eventBusContext);
-        ModeConfiguration actual = context.getModeConfiguration();
-        assertThat(actual, is(modeConfig));
+        assertThat(context.getModeConfiguration(), is(modeConfig));
     }
     
     @Test
     public void assertIsCluster() {
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE), modeConfig,
                 lockContext, eventBusContext);
         assertFalse(context.isCluster());
-        InstanceContext clusterContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
+        InstanceContext clusterContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Integer.MIN_VALUE),
                 new ModeConfiguration("Cluster", null), lockContext, eventBusContext);
         assertTrue(clusterContext.isCluster());
     }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/fixture/WorkerIdGeneratorFixture.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/fixture/WorkerIdGeneratorFixture.java
index e851c61bd2b..d0a34b75dc9 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/fixture/WorkerIdGeneratorFixture.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/fixture/WorkerIdGeneratorFixture.java
@@ -25,10 +25,10 @@ import java.util.Properties;
 @RequiredArgsConstructor
 public final class WorkerIdGeneratorFixture implements WorkerIdGenerator {
     
-    private final long fixtureWorkerId;
+    private final int fixtureWorkerId;
     
     @Override
-    public long generate(final Properties props) {
+    public int generate(final Properties props) {
         return fixtureWorkerId;
     }
 }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
index 221135ddd0b..37601602934 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
@@ -73,7 +73,7 @@ public final class GlobalRulesBuilderTest {
     
     private WorkerIdGenerator createWorkerIdGenerator() {
         WorkerIdGenerator result = mock(WorkerIdGenerator.class);
-        when(result.generate(new Properties())).thenReturn(0L);
+        when(result.generate(new Properties())).thenReturn(0);
         return result;
     }
 }
diff --git a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/WorkerIdEvent.java b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/WorkerIdEvent.java
index 4ce176ce66b..86193002e92 100644
--- a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/WorkerIdEvent.java
+++ b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/WorkerIdEvent.java
@@ -30,5 +30,5 @@ public final class WorkerIdEvent implements GovernanceEvent {
     
     private final String instanceId;
     
-    private final Long workerId;
+    private final Integer workerId;
 }
diff --git a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
index 2fa0f494718..8265af790e8 100644
--- a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
+++ b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusService.java
@@ -32,9 +32,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Optional;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -73,7 +71,7 @@ public final class ComputeNodeStatusService {
      * @param instanceId instance id
      * @param workerId worker id
      */
-    public void persistInstanceWorkerId(final String instanceId, final Long workerId) {
+    public void persistInstanceWorkerId(final String instanceId, final Integer workerId) {
         repository.persistEphemeral(ComputeNode.getInstanceWorkerIdNodePath(instanceId), String.valueOf(workerId));
     }
     
@@ -107,10 +105,10 @@ public final class ComputeNodeStatusService {
      * @param instanceId instance id
      * @return worker id
      */
-    public Optional<Long> loadInstanceWorkerId(final String instanceId) {
+    public Optional<Integer> loadInstanceWorkerId(final String instanceId) {
         try {
             String workerId = repository.getDirectly(ComputeNode.getInstanceWorkerIdNodePath(instanceId));
-            return Strings.isNullOrEmpty(workerId) ? Optional.empty() : Optional.of(Long.valueOf(workerId));
+            return Strings.isNullOrEmpty(workerId) ? Optional.empty() : Optional.of(Integer.valueOf(workerId));
         } catch (final NumberFormatException ex) {
             log.error("Invalid worker id for instance: {}", instanceId);
         }
@@ -155,13 +153,13 @@ public final class ComputeNodeStatusService {
      *
      * @return assigned worker ids
      */
-    public Set<Long> getAssignedWorkerIds() {
-        Set<Long> result = new LinkedHashSet<>();
-        List<String> childrenKeys = repository.getChildrenKeys(ComputeNode.getInstanceWorkerIdRootNodePath());
+    public Collection<Integer> getAssignedWorkerIds() {
+        Collection<Integer> result = new LinkedHashSet<>();
+        Collection<String> childrenKeys = repository.getChildrenKeys(ComputeNode.getInstanceWorkerIdRootNodePath());
         for (String each : childrenKeys) {
             String workerId = repository.getDirectly(ComputeNode.getInstanceWorkerIdNodePath(each));
             if (null != workerId) {
-                result.add(Long.parseLong(workerId));
+                result.add(Integer.parseInt(workerId));
             }
         }
         return result;
diff --git a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java
index a17b49eefa4..beb7170fa47 100644
--- a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java
+++ b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/watcher/ComputeNodeStateChangedWatcher.java
@@ -70,7 +70,7 @@ public final class ComputeNodeStateChangedWatcher implements GovernanceWatcher<G
                 return Optional.of(new StateEvent(instanceId, status));
             }
             if (event.getKey().equals(ComputeNode.getInstanceWorkerIdNodePath(instanceId))) {
-                return Optional.of(new WorkerIdEvent(instanceId, Strings.isNullOrEmpty(event.getValue()) ? null : Long.valueOf(event.getValue())));
+                return Optional.of(new WorkerIdEvent(instanceId, Strings.isNullOrEmpty(event.getValue()) ? null : Integer.valueOf(event.getValue())));
             }
             if (event.getKey().equals(ComputeNode.getInstanceLabelsNodePath(instanceId))) {
                 return Optional.of(new LabelsEvent(instanceId, Strings.isNullOrEmpty(event.getValue()) ? new ArrayList<>() : YamlEngine.unmarshal(event.getValue(), Collection.class)));
diff --git a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGenerator.java b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGenerator.java
index 4964b78e5e2..3ddd625483c 100644
--- a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGenerator.java
+++ b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGenerator.java
@@ -22,17 +22,17 @@ import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.instance.workerid.WorkerIdGenerator;
+import org.apache.shardingsphere.infra.util.exception.ShardingSpherePreconditions;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.RegistryCenter;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.workerid.exception.WorkIdAssignedException;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.workerid.node.WorkerIdNode;
 import org.apache.shardingsphere.mode.repository.cluster.exception.ClusterPersistRepositoryException;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.LinkedList;
 import java.util.Optional;
 import java.util.PriorityQueue;
 import java.util.Properties;
-import java.util.Set;
 
 /**
  * Worker id generator for cluster mode.
@@ -48,36 +48,34 @@ public final class ClusterWorkerIdGenerator implements WorkerIdGenerator {
     private volatile boolean isWarned;
     
     @Override
-    public long generate(final Properties props) {
-        long result = registryCenter.getComputeNodeStatusService().loadInstanceWorkerId(instanceMetaData.getId()).orElseGet(this::reGenerate);
+    public int generate(final Properties props) {
+        int result = registryCenter.getComputeNodeStatusService().loadInstanceWorkerId(instanceMetaData.getId()).orElseGet(this::reGenerate);
         checkIneffectiveConfiguration(result, props);
         return result;
     }
     
-    private Long reGenerate() {
-        Optional<Long> result;
+    private Integer reGenerate() {
+        Optional<Integer> result;
         do {
             result = generateAvailableWorkerId();
         } while (!result.isPresent());
-        Long generatedWorkId = result.get();
+        Integer generatedWorkId = result.get();
         registryCenter.getComputeNodeStatusService().persistInstanceWorkerId(instanceMetaData.getId(), generatedWorkId);
         return generatedWorkId;
     }
     
-    private Optional<Long> generateAvailableWorkerId() {
-        Set<Long> assignedWorkerIds = registryCenter.getComputeNodeStatusService().getAssignedWorkerIds();
-        if (assignedWorkerIds.size() > 1024) {
-            throw new WorkIdAssignedException();
-        }
-        Collection<Long> maxAvailableIds = new ArrayList<>(1024);
+    private Optional<Integer> generateAvailableWorkerId() {
+        Collection<Integer> assignedWorkerIds = registryCenter.getComputeNodeStatusService().getAssignedWorkerIds();
+        ShardingSpherePreconditions.checkState(assignedWorkerIds.size() <= 1024, WorkIdAssignedException::new);
+        Collection<Integer> availableWorkerIds = new LinkedList<>();
         for (int i = 0; i < 1024; i++) {
-            maxAvailableIds.add((long) i);
+            availableWorkerIds.add(i);
         }
-        PriorityQueue<Long> priorityQueue = new PriorityQueue<>(maxAvailableIds);
-        for (Long each : assignedWorkerIds) {
+        PriorityQueue<Integer> priorityQueue = new PriorityQueue<>(availableWorkerIds);
+        for (Integer each : assignedWorkerIds) {
             priorityQueue.remove(each);
         }
-        Long preselectedWorkerId = priorityQueue.poll();
+        Integer preselectedWorkerId = priorityQueue.poll();
         Preconditions.checkState(null != preselectedWorkerId, "Preselected worker-id can not be null.");
         try {
             registryCenter.getRepository().persistExclusiveEphemeral(WorkerIdNode.getWorkerIdGeneratorPath(preselectedWorkerId.toString()), instanceMetaData.getId());
diff --git a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java
index 39409882475..d165d705305 100644
--- a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java
+++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/service/ComputeNodeStatusServiceTest.java
@@ -70,8 +70,8 @@ public final class ComputeNodeStatusServiceTest {
     public void assertPersistInstanceWorkerId() {
         InstanceMetaData instanceMetaData = new ProxyInstanceMetaData("foo_instance_id", 3307);
         final String instanceId = instanceMetaData.getId();
-        new ComputeNodeStatusService(repository).persistInstanceWorkerId(instanceId, 100L);
-        verify(repository).persistEphemeral(ComputeNode.getInstanceWorkerIdNodePath(instanceId), String.valueOf(100L));
+        new ComputeNodeStatusService(repository).persistInstanceWorkerId(instanceId, 100);
+        verify(repository).persistEphemeral(ComputeNode.getInstanceWorkerIdNodePath(instanceId), String.valueOf(100));
     }
     
     @Test
diff --git a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGeneratorTest.java b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGeneratorTest.java
index 3eda691f4ea..bdcc06b42e9 100644
--- a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGeneratorTest.java
+++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/workerid/generator/ClusterWorkerIdGeneratorTest.java
@@ -23,9 +23,9 @@ import org.apache.shardingsphere.mode.manager.cluster.coordinator.RegistryCenter
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.junit.Test;
 import org.mockito.Mockito;
-import org.mockito.internal.util.collections.Sets;
 import org.mockito.stubbing.Answer;
 
+import java.util.Collections;
 import java.util.Optional;
 import java.util.Properties;
 
@@ -44,8 +44,8 @@ public final class ClusterWorkerIdGeneratorTest {
         InstanceMetaData instanceMetaData = mock(InstanceMetaData.class);
         when(instanceMetaData.getId()).thenReturn("foo_id");
         RegistryCenter registryCenter = mock(RegistryCenter.class, RETURNS_DEEP_STUBS);
-        when(registryCenter.getComputeNodeStatusService().loadInstanceWorkerId("foo_id")).thenReturn(Optional.of(10L));
-        assertThat(new ClusterWorkerIdGenerator(registryCenter, instanceMetaData).generate(props), is(10L));
+        when(registryCenter.getComputeNodeStatusService().loadInstanceWorkerId("foo_id")).thenReturn(Optional.of(10));
+        assertThat(new ClusterWorkerIdGenerator(registryCenter, instanceMetaData).generate(props), is(10));
     }
     
     @Test
@@ -57,7 +57,7 @@ public final class ClusterWorkerIdGeneratorTest {
         Mockito.doAnswer((Answer<Object>) invocation -> "foo_id").when(repository).persistEphemeral("/worker_id/0", "foo_id");
         when(registryCenter.getRepository()).thenReturn(repository);
         when(registryCenter.getComputeNodeStatusService().loadInstanceWorkerId("foo_id")).thenReturn(Optional.empty());
-        when(registryCenter.getComputeNodeStatusService().getAssignedWorkerIds()).thenReturn(Sets.newSet(1L));
-        assertThat(new ClusterWorkerIdGenerator(registryCenter, instanceMetaData).generate(new Properties()), is(0L));
+        when(registryCenter.getComputeNodeStatusService().getAssignedWorkerIds()).thenReturn(Collections.singleton(1));
+        assertThat(new ClusterWorkerIdGenerator(registryCenter, instanceMetaData).generate(new Properties()), is(0));
     }
 }
diff --git a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/StateChangedSubscriberTest.java b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/StateChangedSubscriberTest.java
index 85119638d73..347a5c04893 100644
--- a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/StateChangedSubscriberTest.java
+++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/subscriber/StateChangedSubscriberTest.java
@@ -155,8 +155,8 @@ public final class StateChangedSubscriberTest {
     
     @Test
     public void assertRenewInstanceWorkerIdEvent() {
-        subscriber.renew(new WorkerIdEvent(contextManager.getInstanceContext().getInstance().getMetaData().getId(), 0L));
-        assertThat(contextManager.getInstanceContext().getInstance().getWorkerId(), is(0L));
+        subscriber.renew(new WorkerIdEvent(contextManager.getInstanceContext().getInstance().getMetaData().getId(), 0));
+        assertThat(contextManager.getInstanceContext().getInstance().getWorkerId(), is(0));
     }
     
     @Test
diff --git a/mode/type/cluster/repository/provider/zookeeper/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/listener/SessionConnectionListener.java b/mode/type/cluster/repository/provider/zookeeper/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/listener/SessionConnectionListener.java
index 64efcfa5184..217147cfe63 100644
--- a/mode/type/cluster/repository/provider/zookeeper/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/listener/SessionConnectionListener.java
+++ b/mode/type/cluster/repository/provider/zookeeper/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/listener/SessionConnectionListener.java
@@ -74,7 +74,7 @@ public final class SessionConnectionListener implements ConnectionStateListener
     }
     
     private boolean isNeedGenerateWorkerId() {
-        return -1L != instanceContext.getInstance().getWorkerId();
+        return -1 != instanceContext.getInstance().getWorkerId();
     }
     
     @SneakyThrows(InterruptedException.class)
diff --git a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGenerator.java b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGenerator.java
index 73952769f91..0fafbb17c9a 100644
--- a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGenerator.java
+++ b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGenerator.java
@@ -28,7 +28,7 @@ import java.util.Properties;
 public final class StandaloneWorkerIdGenerator implements WorkerIdGenerator {
     
     @Override
-    public long generate(final Properties props) {
+    public int generate(final Properties props) {
         if (null == props) {
             return DEFAULT_WORKER_ID;
         }
@@ -36,7 +36,7 @@ public final class StandaloneWorkerIdGenerator implements WorkerIdGenerator {
         if (null == workerId) {
             return DEFAULT_WORKER_ID;
         }
-        long result = Long.parseLong(workerId.toString());
+        int result = Integer.parseInt(workerId.toString());
         Preconditions.checkState(result <= MAX_WORKER_ID, "%s can not exceed %s", WORKER_ID_KEY, MAX_WORKER_ID);
         return result;
     }
diff --git a/mode/type/standalone/core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGeneratorTest.java b/mode/type/standalone/core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGeneratorTest.java
index b6d83d42061..1ef9ce97de0 100644
--- a/mode/type/standalone/core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGeneratorTest.java
+++ b/mode/type/standalone/core/src/test/java/org/apache/shardingsphere/mode/manager/standalone/workerid/generator/StandaloneWorkerIdGeneratorTest.java
@@ -41,7 +41,7 @@ public final class StandaloneWorkerIdGeneratorTest {
     public void assertGenerateWithProperties() {
         Properties props = new Properties();
         props.setProperty(WorkerIdGenerator.WORKER_ID_KEY, "1");
-        assertThat(new StandaloneWorkerIdGenerator().generate(props), is(1L));
+        assertThat(new StandaloneWorkerIdGenerator().generate(props), is(1));
     }
     
     @Test(expected = IllegalStateException.class)
diff --git a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodeInfoHandlerTest.java b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodeInfoHandlerTest.java
index 006b8e4f276..c74499c470e 100644
--- a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodeInfoHandlerTest.java
+++ b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodeInfoHandlerTest.java
@@ -57,7 +57,7 @@ public final class ShowComputeNodeInfoHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(2), is(3309));
         assertThat(data.get(3), is("OK"));
         assertThat(data.get(4), is("Standalone"));
-        assertThat(data.get(5), is(0L));
+        assertThat(data.get(5), is(0));
         assertThat(data.get(6), is(""));
     }
     
@@ -66,7 +66,7 @@ public final class ShowComputeNodeInfoHandlerTest extends ProxyContextRestorer {
         when(result.getInstance().getMetaData()).thenReturn(new ProxyInstanceMetaData("127.0.0.1@3309", "127.0.0.1@3309"));
         when(result.getInstance().getState()).thenReturn(new StateContext());
         when(result.getModeConfiguration()).thenReturn(new ModeConfiguration("Standalone", new StandalonePersistRepositoryConfiguration("H2", new Properties())));
-        when(result.getInstance().getWorkerId()).thenReturn(0L);
+        when(result.getInstance().getWorkerId()).thenReturn(0);
         return result;
     }
 }
diff --git a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodesHandlerTest.java b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodesHandlerTest.java
index d6a4ee173c9..7ce39ac0bc8 100644
--- a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodesHandlerTest.java
+++ b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowComputeNodesHandlerTest.java
@@ -61,7 +61,7 @@ public final class ShowComputeNodesHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(2), is(3308));
         assertThat(data.get(3), is("OK"));
         assertThat(data.get(4), is("Standalone"));
-        assertThat(data.get(5), is(0L));
+        assertThat(data.get(5), is(0));
         assertThat(data.get(6), is(""));
     }
     
@@ -81,7 +81,7 @@ public final class ShowComputeNodesHandlerTest extends ProxyContextRestorer {
         assertThat(data.get(2), is(3309));
         assertThat(data.get(3), is("OK"));
         assertThat(data.get(4), is("Cluster"));
-        assertThat(data.get(5), is(1L));
+        assertThat(data.get(5), is(1));
         assertThat(data.get(6), is(""));
     }
     
@@ -90,7 +90,7 @@ public final class ShowComputeNodesHandlerTest extends ProxyContextRestorer {
         when(result.getInstance().getMetaData()).thenReturn(new ProxyInstanceMetaData("127.0.0.1@3308", "127.0.0.1@3308"));
         when(result.getInstance().getState()).thenReturn(new StateContext());
         when(result.getModeConfiguration()).thenReturn(new ModeConfiguration("Standalone", new StandalonePersistRepositoryConfiguration("H2", new Properties())));
-        when(result.getInstance().getWorkerId()).thenReturn(0L);
+        when(result.getInstance().getWorkerId()).thenReturn(0);
         return result;
     }
     
@@ -100,7 +100,7 @@ public final class ShowComputeNodesHandlerTest extends ProxyContextRestorer {
         ComputeNodeInstance computeNodeInstance = mock(ComputeNodeInstance.class, RETURNS_DEEP_STUBS);
         when(computeNodeInstance.getMetaData()).thenReturn(new ProxyInstanceMetaData("127.0.0.1@3309", "127.0.0.1@3309"));
         when(computeNodeInstance.getState()).thenReturn(new StateContext());
-        when(computeNodeInstance.getWorkerId()).thenReturn(1L);
+        when(computeNodeInstance.getWorkerId()).thenReturn(1);
         when(result.getAllClusterInstances()).thenReturn(Collections.singleton(computeNodeInstance));
         return result;
     }
diff --git a/test/integration-test/sql-parser/src/test/java/org/apache/shardingsphere/sql/parser/result/SQLParserLogResultProcessor.java b/test/integration-test/sql-parser/src/test/java/org/apache/shardingsphere/sql/parser/result/SQLParserLogResultProcessor.java
index 9392563421e..6c508e51891 100644
--- a/test/integration-test/sql-parser/src/test/java/org/apache/shardingsphere/sql/parser/result/SQLParserLogResultProcessor.java
+++ b/test/integration-test/sql-parser/src/test/java/org/apache/shardingsphere/sql/parser/result/SQLParserLogResultProcessor.java
@@ -27,7 +27,7 @@ public class SQLParserLogResultProcessor implements SQLParserResultProcessor {
     private final String type = "LOG";
     
     public SQLParserLogResultProcessor(final String databaseType) {
-    
+        
     }
     
     @Override