You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by to...@apache.org on 2022/05/29 16:40:37 UTC

[shardingsphere] branch master updated: refactor DistSQL of instance label and status with instance id (#18043)

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

totalo 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 0f8ffc35a6a refactor DistSQL of instance label and status with instance id (#18043)
0f8ffc35a6a is described below

commit 0f8ffc35a6a87af4fe9af49a0c184220ac2a7497
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Mon May 30 00:40:30 2022 +0800

    refactor DistSQL of instance label and status with instance id (#18043)
---
 .../src/main/antlr4/imports/RALStatement.g4        | 16 +++-----
 .../core/common/CommonDistSQLStatementVisitor.java | 44 ++++------------------
 .../common/updatable/LabelInstanceStatement.java   |  4 +-
 .../updatable/SetInstanceStatusStatement.java      |  4 +-
 .../common/updatable/UnlabelInstanceStatement.java |  4 +-
 .../event/ComputeNodeStatusChangedEvent.java       |  4 +-
 .../subscriber/ComputeNodeStatusSubscriber.java    |  3 +-
 .../ral/common/updatable/LabelInstanceHandler.java |  4 +-
 .../common/updatable/SetInstanceStatusHandler.java | 31 +++++++--------
 .../common/updatable/UnlabelInstanceHandler.java   |  4 +-
 .../updatable/LabelInstanceStatementAssert.java    |  4 +-
 .../updatable/UnlabelInstanceStatementAssert.java  |  4 +-
 .../ral/LabelInstanceStatementTestCase.java        |  7 +---
 .../ral/UnlabelInstanceStatementTestCase.java      |  7 +---
 .../src/main/resources/case/ral/common.xml         | 12 +++---
 .../main/resources/sql/supported/ral/common.xml    | 12 +++---
 16 files changed, 53 insertions(+), 111 deletions(-)

diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
index e259e5f2e5c..df7021d7402 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
@@ -36,11 +36,11 @@ alterInstance
     ;
 
 enableInstance
-    : ENABLE INSTANCE (instanceId | instanceDefination)
+    : ENABLE INSTANCE instanceId
     ;
 
 disableInstance
-    : DISABLE INSTANCE (instanceId | instanceDefination)
+    : DISABLE INSTANCE instanceId
     ;
 
 showInstance
@@ -100,11 +100,11 @@ dropTrafficRule
     ;
 
 labelInstance
-    : (LABEL | RELABEL) INSTANCE (instanceDefination | instanceId) WITH label (COMMA label)*
+    : (LABEL | RELABEL) INSTANCE instanceId WITH label (COMMA label)*
     ;
 
 unlabelInstance
-    : UNLABEL INSTANCE (instanceDefination | instanceId) (WITH label (COMMA label)*)?
+    : UNLABEL INSTANCE instanceId (WITH label (COMMA label)*)?
     ;
 
 countInstanceRules
@@ -176,15 +176,11 @@ variableValues
     ;
 
 variableValue
-    : IDENTIFIER | STRING | (MINUS)? INT | TRUE | FALSE | instanceId
-    ;
-
-instanceDefination
-    : IP EQ ip COMMA PORT EQ port
+    : IDENTIFIER | STRING | (MINUS)? INT | TRUE | FALSE
     ;
 
 instanceId
-    : ip AT port
+    : IDENTIFIER | STRING
     ;
 
 refreshScope
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
index d443b733154..7f339d9248a 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
@@ -47,7 +47,6 @@ import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementPa
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.ExportDatabaseConfigurationContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.FromSegmentContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.ImportDatabaseConfigurationContext;
-import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.InstanceDefinationContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.InstanceIdContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.LabelDefinitionContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.LabelInstanceContext;
@@ -187,55 +186,28 @@ public final class CommonDistSQLStatementVisitor extends CommonDistSQLStatementB
     
     @Override
     public ASTNode visitEnableInstance(final EnableInstanceContext ctx) {
-        return buildSetInstanceStatusStatement(ctx.ENABLE().getText().toUpperCase(), ctx.instanceDefination(), ctx.instanceId());
+        return buildSetInstanceStatusStatement(ctx.ENABLE().getText().toUpperCase(), ctx.instanceId());
     }
     
     @Override
     public ASTNode visitDisableInstance(final DisableInstanceContext ctx) {
-        return buildSetInstanceStatusStatement(ctx.DISABLE().getText().toUpperCase(), ctx.instanceDefination(), ctx.instanceId());
+        return buildSetInstanceStatusStatement(ctx.DISABLE().getText().toUpperCase(), ctx.instanceId());
     }
     
     @Override
     public ASTNode visitLabelInstance(final LabelInstanceContext ctx) {
-        String ip;
-        String port;
-        if (null != ctx.instanceDefination()) {
-            ip = getIdentifierValue(ctx.instanceDefination().ip());
-            port = getIdentifierValue(ctx.instanceDefination().port());
-        } else {
-            ip = getIdentifierValue(ctx.instanceId().ip());
-            port = getIdentifierValue(ctx.instanceId().port());
-        }
         Collection<String> labels = ctx.label().stream().map(this::getIdentifierValue).collect(Collectors.toList());
-        return new LabelInstanceStatement(ctx.RELABEL() != null, ip, port, labels);
+        return new LabelInstanceStatement(ctx.RELABEL() != null, getIdentifierValue(ctx.instanceId()), labels);
     }
     
     @Override
     public ASTNode visitUnlabelInstance(final UnlabelInstanceContext ctx) {
-        String ip;
-        String port;
-        if (null != ctx.instanceDefination()) {
-            ip = getIdentifierValue(ctx.instanceDefination().ip());
-            port = getIdentifierValue(ctx.instanceDefination().port());
-        } else {
-            ip = getIdentifierValue(ctx.instanceId().ip());
-            port = getIdentifierValue(ctx.instanceId().port());
-        }
         Collection<String> labels = ctx.label().stream().map(this::getIdentifierValue).collect(Collectors.toList());
-        return new UnlabelInstanceStatement(ip, port, labels);
-    }
-    
-    private SetInstanceStatusStatement buildSetInstanceStatusStatement(final String status, final InstanceDefinationContext instanceDefinationContext, final InstanceIdContext instanceIdContext) {
-        String ip;
-        String port;
-        if (null != instanceDefinationContext) {
-            ip = getIdentifierValue(instanceDefinationContext.ip());
-            port = getIdentifierValue(instanceDefinationContext.port());
-        } else {
-            ip = getIdentifierValue(instanceIdContext.ip());
-            port = getIdentifierValue(instanceIdContext.port());
-        }
-        return new SetInstanceStatusStatement(status, ip, port);
+        return new UnlabelInstanceStatement(getIdentifierValue(ctx.instanceId()), labels);
+    }
+    
+    private SetInstanceStatusStatement buildSetInstanceStatusStatement(final String status, final InstanceIdContext instanceIdContext) {
+        return new SetInstanceStatusStatement(status, getIdentifierValue(instanceIdContext));
     }
     
     @Override
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/LabelInstanceStatement.java b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/LabelInstanceStatement.java
index f92d585b10b..52c8820487a 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/LabelInstanceStatement.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/LabelInstanceStatement.java
@@ -32,9 +32,7 @@ public class LabelInstanceStatement extends UpdatableRALStatement {
     
     private final boolean overwrite;
     
-    private final String ip;
-    
-    private final String port;
+    private final String instanceId;
     
     private final Collection<String> labels;
 }
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/SetInstanceStatusStatement.java b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/SetInstanceStatusStatement.java
index eaad8dd282d..546e6780c96 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/SetInstanceStatusStatement.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/SetInstanceStatusStatement.java
@@ -30,7 +30,5 @@ public class SetInstanceStatusStatement extends UpdatableRALStatement {
     
     private final String status;
     
-    private final String ip;
-    
-    private final String port;
+    private final String instanceId;
 }
diff --git a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/UnlabelInstanceStatement.java b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/UnlabelInstanceStatement.java
index 03f5da7281b..97925559e01 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/UnlabelInstanceStatement.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/updatable/UnlabelInstanceStatement.java
@@ -30,9 +30,7 @@ import java.util.Collection;
 @RequiredArgsConstructor
 public class UnlabelInstanceStatement extends UpdatableRALStatement {
     
-    private final String ip;
-    
-    private final String port;
+    private final String instanceId;
     
     private final Collection<String> labels;
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/ComputeNodeStatusChangedEvent.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/ComputeNodeStatusChangedE [...]
index 2e6f27824a2..8f443669bda 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/ComputeNodeStatusChangedEvent.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/event/ComputeNodeStatusChangedEvent.java
@@ -30,7 +30,5 @@ public final class ComputeNodeStatusChangedEvent {
     
     private final ComputeNodeStatus status;
     
-    private final String ip;
-    
-    private final String port;
+    private final String instanceId;
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/subscriber/ComputeNodeStatusSubscriber.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/subscriber/ComputeNodeStatus [...]
index e22be627165..6eb6c733c1f 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/subscriber/ComputeNodeStatusSubscriber.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/compute/subscriber/ComputeNodeStatusSubscriber.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.stat
 import com.google.common.base.Strings;
 import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.instance.definition.InstanceId;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.RegistryCenter;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.compute.ComputeNodeStatus;
@@ -56,7 +55,7 @@ public final class ComputeNodeStatusSubscriber {
      */
     @Subscribe
     public void update(final ComputeNodeStatusChangedEvent event) {
-        String computeStatusNodePath = ComputeNode.getInstanceStatusNodePath(new InstanceId(event.getIp(), String.valueOf(event.getPort())).getId());
+        String computeStatusNodePath = ComputeNode.getInstanceStatusNodePath(event.getInstanceId());
         String yamlContext = repository.get(computeStatusNodePath);
         Collection<String> status = Strings.isNullOrEmpty(yamlContext) ? new ArrayList<>() : YamlEngine.unmarshal(yamlContext, Collection.class);
         if (event.getStatus() == ComputeNodeStatus.CIRCUIT_BREAK) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/LabelInstanceHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/LabelInstanceHandler.java
index 8001b0318b5..b4638131b9d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/LabelInstanceHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/LabelInstanceHandler.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.LabelInstanceStatement;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
-import org.apache.shardingsphere.infra.instance.definition.InstanceId;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.compute.event.LabelsChangedEvent;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -44,8 +43,7 @@ public final class LabelInstanceHandler extends UpdatableRALBackendHandler<Label
         if (null == persistService || null == persistService.getRepository() || persistService.getRepository() instanceof StandalonePersistRepository) {
             throw new UnsupportedOperationException("Labels can only be added in cluster mode");
         }
-        // TODO use instance id
-        String instanceId = new InstanceId(sqlStatement.getIp(), String.valueOf(sqlStatement.getPort())).getId();
+        String instanceId = sqlStatement.getInstanceId();
         Optional<ComputeNodeInstance> computeNodeInstance = contextManager.getInstanceContext().getComputeNodeInstanceById(instanceId);
         if (computeNodeInstance.isPresent()) {
             Collection<String> labels = new LinkedHashSet<>(sqlStatement.getLabels());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetInstanceStatusHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetInstanceStatusHandler.java
index 653d009bbf5..1c2aac44bf6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetInstanceStatusHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetInstanceStatusHandler.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.updatabl
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.SetInstanceStatusStatement;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.instance.definition.InstanceId;
 import org.apache.shardingsphere.infra.state.StateType;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.compute.ComputeNodeStatus;
@@ -36,33 +35,31 @@ public final class SetInstanceStatusHandler extends UpdatableRALBackendHandler<S
         if (!"Cluster".equals(contextManager.getInstanceContext().getModeConfiguration().getType())) {
             throw new UnsupportedOperationException("Only allowed in cluster mode");
         }
-        // TODO use instance id
-        InstanceId operationInstanceId = new InstanceId(sqlStatement.getIp(), String.valueOf(sqlStatement.getPort()));
+        String instanceId = sqlStatement.getInstanceId();
         boolean isDisable = "DISABLE".equals(sqlStatement.getStatus());
         if (isDisable) {
-            checkDisablingIsValid(contextManager, operationInstanceId);
+            checkDisablingIsValid(contextManager, instanceId);
         } else {
-            checkEnablingIsValid(contextManager, operationInstanceId);
+            checkEnablingIsValid(contextManager, instanceId);
         }
-        ShardingSphereEventBus.getInstance().post(new ComputeNodeStatusChangedEvent(isDisable ? ComputeNodeStatus.CIRCUIT_BREAK : ComputeNodeStatus.ONLINE,
-                sqlStatement.getIp(), sqlStatement.getPort()));
+        ShardingSphereEventBus.getInstance().post(new ComputeNodeStatusChangedEvent(isDisable ? ComputeNodeStatus.CIRCUIT_BREAK : ComputeNodeStatus.ONLINE, instanceId));
     }
     
-    private void checkEnablingIsValid(final ContextManager contextManager, final InstanceId operationInstanceId) {
-        if (!contextManager.getInstanceContext().getComputeNodeInstanceById(operationInstanceId.getId()).isPresent()) {
-            throw new UnsupportedOperationException(String.format("`%s` does not exist", operationInstanceId.getId()));
+    private void checkEnablingIsValid(final ContextManager contextManager, final String instanceId) {
+        if (!contextManager.getInstanceContext().getComputeNodeInstanceById(instanceId).isPresent()) {
+            throw new UnsupportedOperationException(String.format("`%s` does not exist", instanceId));
         }
     }
     
-    private void checkDisablingIsValid(final ContextManager contextManager, final InstanceId operationInstanceId) {
-        if (contextManager.getInstanceContext().getInstance().getCurrentInstanceId().equals(operationInstanceId.getId())) {
-            throw new UnsupportedOperationException(String.format("`%s` is the currently in use instance and cannot be disabled", operationInstanceId.getId()));
+    private void checkDisablingIsValid(final ContextManager contextManager, final String instanceId) {
+        if (contextManager.getInstanceContext().getInstance().getCurrentInstanceId().equals(instanceId)) {
+            throw new UnsupportedOperationException(String.format("`%s` is the currently in use instance and cannot be disabled", instanceId));
         }
-        if (!contextManager.getInstanceContext().getComputeNodeInstanceById(operationInstanceId.getId()).isPresent()) {
-            throw new UnsupportedOperationException(String.format("`%s` does not exist", operationInstanceId.getId()));
+        if (!contextManager.getInstanceContext().getComputeNodeInstanceById(instanceId).isPresent()) {
+            throw new UnsupportedOperationException(String.format("`%s` does not exist", instanceId));
         }
-        if (contextManager.getInstanceContext().getComputeNodeInstanceById(operationInstanceId.getId()).get().getState().getCurrentState() == StateType.CIRCUIT_BREAK) {
-            throw new UnsupportedOperationException(String.format("`%s` compute node has been disabled", operationInstanceId.getId()));
+        if (contextManager.getInstanceContext().getComputeNodeInstanceById(instanceId).get().getState().getCurrentState() == StateType.CIRCUIT_BREAK) {
+            throw new UnsupportedOperationException(String.format("`%s` compute node has been disabled", instanceId));
         }
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/UnlabelInstanceHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/UnlabelInstanceHandler.java
index 95ec88c6f96..e59b6e2bcc9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/UnlabelInstanceHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/UnlabelInstanceHandler.java
@@ -21,7 +21,6 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.updatable.U
 import org.apache.shardingsphere.infra.distsql.exception.DistSQLException;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
-import org.apache.shardingsphere.infra.instance.definition.InstanceId;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.compute.event.LabelsChangedEvent;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
@@ -46,8 +45,7 @@ public final class UnlabelInstanceHandler extends UpdatableRALBackendHandler<Unl
         if (null == persistService || null == persistService.getRepository() || persistService.getRepository() instanceof StandalonePersistRepository) {
             throw new UnsupportedOperationException("Labels can only be removed in cluster mode");
         }
-        // TODO use instance id
-        String instanceId = new InstanceId(sqlStatement.getIp(), String.valueOf(sqlStatement.getPort())).getId();
+        String instanceId = sqlStatement.getInstanceId();
         Optional<ComputeNodeInstance> computeNodeInstance = contextManager.getInstanceContext().getComputeNodeInstanceById(instanceId);
         if (computeNodeInstance.isPresent()) {
             Collection<String> labels = new LinkedHashSet<>(computeNodeInstance.get().getLabels());
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/LabelInstanceStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/LabelInstanceStatementAssert.java
index f29c561c543..d44bc901b7d 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/LabelInstanceStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/LabelInstanceStatementAssert.java
@@ -47,9 +47,7 @@ public final class LabelInstanceStatementAssert {
         } else {
             assertNotNull(assertContext.getText("Actual statement should exist."), actual);
             assertThat(actual.isOverwrite(), is(expected.isOverwrite()));
-            assertThat(actual.getIp(), is(expected.getIp()));
-            assertThat(actual.getPort(), is(expected.getPort()));
-            assertThat(actual.getPort(), is(actual.getPort()));
+            assertThat(actual.getInstanceId(), is(expected.getInstanceId()));
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/UnlabelInstanceStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/UnlabelInstanceStatementAssert.java
index b1027f49db3..f43f274b9ea 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/UnlabelInstanceStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/updatable/UnlabelInstanceStatementAssert.java
@@ -46,9 +46,7 @@ public final class UnlabelInstanceStatementAssert {
             assertNull(assertContext.getText("Actual statement should not exist."), actual);
         } else {
             assertNotNull(assertContext.getText("Actual statement should exist."), actual);
-            assertThat(actual.getIp(), is(expected.getIp()));
-            assertThat(actual.getPort(), is(expected.getPort()));
-            assertThat(actual.getPort(), is(actual.getPort()));
+            assertThat(actual.getInstanceId(), is(expected.getInstanceId()));
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/LabelInstanceStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/LabelInstanceStatementTestCase.java
index eed76c17d13..325321392ec 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/LabelInstanceStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/LabelInstanceStatementTestCase.java
@@ -35,11 +35,8 @@ public final class LabelInstanceStatementTestCase extends SQLParserTestCase {
     @XmlAttribute(name = "overwrite")
     private boolean overwrite;
     
-    @XmlAttribute(name = "ip")
-    private String ip;
-    
-    @XmlAttribute(name = "port")
-    private String port;
+    @XmlAttribute(name = "instance-id")
+    private String instanceId;
     
     @XmlElement(name = "label")
     private Collection<String> labels;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/UnlabelInstanceStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/UnlabelInstanceStatementTestCase.java
index 5edb3f52ba2..c58449a7e31 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/UnlabelInstanceStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/UnlabelInstanceStatementTestCase.java
@@ -32,11 +32,8 @@ import java.util.Collection;
 @Setter
 public final class UnlabelInstanceStatementTestCase extends SQLParserTestCase {
     
-    @XmlAttribute(name = "ip")
-    private String ip;
-    
-    @XmlAttribute(name = "port")
-    private String port;
+    @XmlAttribute(name = "instance-id")
+    private String instanceId;
     
     @XmlElement(name = "label")
     private Collection<String> labels;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
index 96f83c8cb3c..762f07fa8a6 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
@@ -86,24 +86,24 @@
             </load-balancer>
         </rule>
     </alter-traffic-rule>
-    <label-instance sql-case-id="label-instance" ip="127.0.0.1" port="3309" overwrite="false">
+    <label-instance sql-case-id="label-instance" instance-id="instance_label" overwrite="false">
         <label>OLAP_1</label>
         <label>OLAP_2</label>
     </label-instance>
-    <label-instance sql-case-id="relabel-instance" ip="127.0.0.1" port="3309" overwrite="true">
+    <label-instance sql-case-id="relabel-instance" instance-id="instance_relabel" overwrite="true">
         <label>OLAP_1</label>
         <label>OLAP_2</label>
     </label-instance>
-    <unlabel-instance sql-case-id="unlabel-instance" ip="127.0.0.1" port="3309">
+    <unlabel-instance sql-case-id="unlabel-instance" instance-id="instance_unlabel">
         <label>OLAP_1</label>
     </unlabel-instance>
-    <unlabel-instance sql-case-id="unlabel-instance-without-label" ip="127.0.0.1" port="3309">
+    <unlabel-instance sql-case-id="unlabel-instance-without-label" instance-id="instance_unlabel_2">
     </unlabel-instance>
     <prepare-distsql sql-case-id="prepare-distsql" />
     <apply-distsql sql-case-id="apply-distsql" />
     <discard-distsql sql-case-id="discard-distsql" />
-    <alter-instance sql-case-id="alter-instance-xa-recovery-nodes" instance-id="1.2.3.4@3307" key="XA_RECOVERY_NODES" value="1.2.3.4@3308" />
-    <alter-instance sql-case-id="alter-instance-xa-recovery-nodes-with-multiple-values" instance-id="1.2.3.4@3307" key="XA_RECOVERY_NODES" value="1.2.3.4@3308,1.2.3.4@3307" />
+    <alter-instance sql-case-id="alter-instance-xa-recovery-nodes" instance-id="instance_xa_recovery" key="XA_RECOVERY_NODES" value="instance_1" />
+    <alter-instance sql-case-id="alter-instance-xa-recovery-nodes-with-multiple-values" instance-id="instance_xa_recovery_multiple" key="XA_RECOVERY_NODES" value="instance_1,instance_2" />
     <import-database-config sql-case-id="import-database-config">
         <filePath>/yaml/config-sharding.yaml</filePath>
     </import-database-config>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
index 59e9f8ebc50..9c4e568b2b9 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
@@ -46,14 +46,14 @@
     <distsql-case id="export-database-config" value="EXPORT DATABASE CONFIG FROM database_name" />
     <distsql-case id="create-traffic-rule" value="CREATE TRAFFIC RULE sql_match_traffic (LABELS(OLTP),TRAFFIC_ALGORITHM(TYPE(NAME=SQL_MATCH,PROPERTIES('sql' = 'SELECT * FROM t_order WHERE order_id = ?; UPDATE t_order SET order_id = ?;'))), LOAD_BALANCER(TYPE(NAME=RANDOM, PROPERTIES('key'='value'))) )" />
     <distsql-case id="alter-traffic-rule" value="ALTER TRAFFIC RULE sql_match_traffic (LABELS(OLTP),TRAFFIC_ALGORITHM(TYPE(NAME=SQL_MATCH,PROPERTIES('sql' = 'SELECT * FROM t_order WHERE order_id = ?; UPDATE t_order SET order_id = ?;'))), LOAD_BALANCER(TYPE(NAME=RANDOM, PROPERTIES('key'='value'))) )" />
-    <distsql-case id="label-instance" value="LABEL INSTANCE 127.0.0.1@3309 WITH OLAP_1,OLAP_2" />
-    <distsql-case id="relabel-instance" value="RELABEL INSTANCE 127.0.0.1@3309 WITH OLAP_1,OLAP_2" />
-    <distsql-case id="unlabel-instance" value="UNLABEL INSTANCE 127.0.0.1@3309 WITH OLAP_1" />
-    <distsql-case id="unlabel-instance-without-label" value="UNLABEL INSTANCE 127.0.0.1@3309" />
+    <distsql-case id="label-instance" value="LABEL INSTANCE instance_label WITH OLAP_1,OLAP_2" />
+    <distsql-case id="relabel-instance" value="RELABEL INSTANCE instance_relabel WITH OLAP_1,OLAP_2" />
+    <distsql-case id="unlabel-instance" value="UNLABEL INSTANCE instance_unlabel WITH OLAP_1" />
+    <distsql-case id="unlabel-instance-without-label" value="UNLABEL INSTANCE instance_unlabel_2" />
     <distsql-case id="prepare-distsql" value="PREPARE DISTSQL" />
     <distsql-case id="apply-distsql" value="APPLY DISTSQL" />
     <distsql-case id="discard-distsql" value="DISCARD DISTSQL" />
-    <distsql-case id="alter-instance-xa-recovery-nodes" value="ALTER INSTANCE 1.2.3.4@3307 SET XA_RECOVERY_NODES=1.2.3.4@3308" />
-    <distsql-case id="alter-instance-xa-recovery-nodes-with-multiple-values" value="ALTER INSTANCE 1.2.3.4@3307 SET XA_RECOVERY_NODES=1.2.3.4@3308,1.2.3.4@3307 " />
+    <distsql-case id="alter-instance-xa-recovery-nodes" value="ALTER INSTANCE instance_xa_recovery SET XA_RECOVERY_NODES=instance_1" />
+    <distsql-case id="alter-instance-xa-recovery-nodes-with-multiple-values" value="ALTER INSTANCE instance_xa_recovery_multiple SET XA_RECOVERY_NODES=instance_1,instance_2 " />
     <distsql-case id="import-database-config" value="IMPORT DATABASE CONFIG FILE='/yaml/config-sharding.yaml'" />
 </sql-cases>