You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2023/02/04 12:26:21 UTC

[shardingsphere] branch master updated: Replace `ShowMigrationRuleResultSet` with `ShowMigrationRuleExecutor` (#24000)

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

jianglongtao 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 a916e11a31d Replace `ShowMigrationRuleResultSet` with `ShowMigrationRuleExecutor` (#24000)
a916e11a31d is described below

commit a916e11a31d77c43763cba162f2fc783bed153cd
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Sun Feb 5 01:26:13 2023 +1300

    Replace `ShowMigrationRuleResultSet` with `ShowMigrationRuleExecutor` (#24000)
---
 ...sultSet.java => ShowMigrationRuleExecutor.java} | 33 +++++-------------
 ....distsql.handler.ral.query.QueryableRALExecutor |  1 +
 ...here.distsql.handler.resultset.DistSQLResultSet | 18 ----------
 .../queryable/ShowMigrationRuleExecutorTest.java   | 40 ++++++++++++++++++++++
 4 files changed, 49 insertions(+), 43 deletions(-)

diff --git a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleResultSet.java b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutor.java
similarity index 66%
rename from proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleResultSet.java
rename to proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutor.java
index 447226206a3..2e26da69ef1 100644
--- a/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleResultSet.java
+++ b/proxy/backend/src/main/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutor.java
@@ -21,35 +21,28 @@ import com.google.gson.Gson;
 import org.apache.shardingsphere.data.pipeline.api.config.process.PipelineProcessConfiguration;
 import org.apache.shardingsphere.data.pipeline.core.api.InventoryIncrementalJobAPI;
 import org.apache.shardingsphere.data.pipeline.core.api.PipelineJobAPI;
-import org.apache.shardingsphere.distsql.handler.resultset.DatabaseDistSQLResultSet;
+import org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.queryable.ShowMigrationRuleStatement;
-import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
 import java.util.LinkedList;
 
 /**
- * Result set for show migration rule.
+ * Show migration rule executor.
  */
-public final class ShowMigrationRuleResultSet implements DatabaseDistSQLResultSet {
+public final class ShowMigrationRuleExecutor implements QueryableRALExecutor<ShowMigrationRuleStatement> {
     
     private static final Gson GSON = new Gson();
     
-    private Iterator<Collection<Object>> data;
-    
     @Override
-    public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
+    public Collection<LocalDataQueryResultRow> getRows(final ShowMigrationRuleStatement sqlStatement) {
         PipelineProcessConfiguration processConfig = ((InventoryIncrementalJobAPI) TypedSPILoader.getService(PipelineJobAPI.class, "MIGRATION")).showProcessConfiguration();
-        Collection<Object> row = new LinkedList<>();
-        row.add(getString(processConfig.getRead()));
-        row.add(getString(processConfig.getWrite()));
-        row.add(getString(processConfig.getStreamChannel()));
-        data = Collections.singletonList(row).iterator();
+        Collection<LocalDataQueryResultRow> result = new LinkedList<>();
+        result.add(new LocalDataQueryResultRow(getString(processConfig.getRead()), getString(processConfig.getWrite()), getString(processConfig.getStreamChannel())));
+        return result;
     }
     
     private String getString(final Object obj) {
@@ -61,16 +54,6 @@ public final class ShowMigrationRuleResultSet implements DatabaseDistSQLResultSe
         return Arrays.asList("read", "write", "stream_channel");
     }
     
-    @Override
-    public boolean next() {
-        return data.hasNext();
-    }
-    
-    @Override
-    public Collection<Object> getRowData() {
-        return data.next();
-    }
-    
     @Override
     public String getType() {
         return ShowMigrationRuleStatement.class.getName();
diff --git a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor b/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
index abdde6ca7a6..8819524c883 100644
--- a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
+++ b/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
@@ -24,3 +24,4 @@ org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowDistVa
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowDistVariablesExecutor
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowStatusFromReadwriteSplittingRulesExecutor
 org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowTableMetaDataExecutor
+org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowMigrationRuleExecutor
diff --git a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet b/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
deleted file mode 100644
index 7a91893abbf..00000000000
--- a/proxy/backend/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# 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.
-#
-
-org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable.ShowMigrationRuleResultSet
diff --git a/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutorTest.java b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutorTest.java
new file mode 100644
index 00000000000..fa99af68969
--- /dev/null
+++ b/proxy/backend/src/test/java/org/apache/shardingsphere/proxy/backend/handler/distsql/ral/queryable/ShowMigrationRuleExecutorTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.handler.distsql.ral.queryable;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Iterator;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public final class ShowMigrationRuleExecutorTest {
+    
+    @Test
+    public void assertGetColumnNames() {
+        ShowMigrationRuleExecutor executor = new ShowMigrationRuleExecutor();
+        Collection<String> columns = executor.getColumnNames();
+        assertThat(columns.size(), is(3));
+        Iterator<String> iterator = columns.iterator();
+        assertThat(iterator.next(), is("read"));
+        assertThat(iterator.next(), is("write"));
+        assertThat(iterator.next(), is("stream_channel"));
+    }
+}