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/03 11:20:34 UTC

[shardingsphere] branch master updated: Replace `ShowMigrationListResultSet` with `ShowMigrationListExecutor` (#23971)

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 312c6e16dfe Replace `ShowMigrationListResultSet` with `ShowMigrationListExecutor` (#23971)
312c6e16dfe is described below

commit 312c6e16dfe6ea0c4ed5dac6be14a7a6fa736c76
Author: Zichao <57...@users.noreply.github.com>
AuthorDate: Sat Feb 4 00:20:26 2023 +1300

    Replace `ShowMigrationListResultSet` with `ShowMigrationListExecutor` (#23971)
---
 .../ral/scaling/QueryableScalingRALStatement.java  |  4 +-
 ...sultSet.java => ShowMigrationListExecutor.java} | 42 +++++----------------
 ...distsql.handler.ral.query.QueryableRALExecutor} |  6 +--
 ...here.distsql.handler.resultset.DistSQLResultSet |  1 -
 .../query/ShowMigrationListExecutorTest.java       | 43 ++++++++++++++++++++++
 5 files changed, 55 insertions(+), 41 deletions(-)

diff --git a/distsql/statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/scaling/QueryableScalingRALStatement.java b/distsql/statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/scaling/QueryableScalingRALStatement.java
index f297acc15df..7df59b1bb84 100644
--- a/distsql/statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/scaling/QueryableScalingRALStatement.java
+++ b/distsql/statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/scaling/QueryableScalingRALStatement.java
@@ -17,10 +17,10 @@
 
 package org.apache.shardingsphere.distsql.parser.statement.ral.scaling;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.ScalingRALStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.QueryableRALStatement;
 
 /**
  * Queryable RAL statement.
  */
-public abstract class QueryableScalingRALStatement extends ScalingRALStatement {
+public abstract class QueryableScalingRALStatement extends QueryableRALStatement {
 }
diff --git a/kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListResultSet.java b/kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutor.java
similarity index 50%
rename from kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListResultSet.java
rename to kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutor.java
index 7d9226c19f3..17818d37b24 100644
--- a/kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListResultSet.java
+++ b/kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutor.java
@@ -17,43 +17,29 @@
 
 package org.apache.shardingsphere.migration.distsql.handler.query;
 
-import org.apache.shardingsphere.data.pipeline.api.pojo.PipelineJobMetaData;
 import org.apache.shardingsphere.data.pipeline.api.pojo.TableBasedPipelineJobInfo;
 import org.apache.shardingsphere.data.pipeline.scenario.migration.api.impl.MigrationJobAPI;
-import org.apache.shardingsphere.distsql.handler.resultset.DatabaseDistSQLResultSet;
-import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor;
+import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
 import org.apache.shardingsphere.migration.distsql.statement.ShowMigrationListStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.stream.Collectors;
 
 /**
- * Result set for show migration list.
+ * Show migration list executor.
  */
-public final class ShowMigrationListResultSet implements DatabaseDistSQLResultSet {
+public final class ShowMigrationListExecutor implements QueryableRALExecutor<ShowMigrationListStatement> {
     
     private final MigrationJobAPI jobAPI = new MigrationJobAPI();
     
-    private Iterator<Collection<Object>> data;
-    
     @Override
-    public void init(final ShardingSphereDatabase database, final SQLStatement sqlStatement) {
-        data = jobAPI.list().stream()
-                .map(each -> {
-                    Collection<Object> result = new LinkedList<>();
-                    PipelineJobMetaData jobMetaData = each.getJobMetaData();
-                    result.add(jobMetaData.getJobId());
-                    result.add(((TableBasedPipelineJobInfo) each).getTable());
-                    result.add(jobMetaData.getJobItemCount());
-                    result.add(jobMetaData.isActive() ? Boolean.TRUE.toString() : Boolean.FALSE.toString());
-                    result.add(jobMetaData.getCreateTime());
-                    result.add(jobMetaData.getStopTime());
-                    return result;
-                }).collect(Collectors.toList()).iterator();
+    public Collection<LocalDataQueryResultRow> getRows(final ShowMigrationListStatement sqlStatement) {
+        return jobAPI.list().stream().map(each -> new LocalDataQueryResultRow(each.getJobMetaData().getJobId(),
+                ((TableBasedPipelineJobInfo) each).getTable(), each.getJobMetaData().getJobItemCount(),
+                each.getJobMetaData().isActive() ? Boolean.TRUE.toString() : Boolean.FALSE.toString(),
+                each.getJobMetaData().getCreateTime(), each.getJobMetaData().getStopTime())).collect(Collectors.toList());
     }
     
     @Override
@@ -61,16 +47,6 @@ public final class ShowMigrationListResultSet implements DatabaseDistSQLResultSe
         return Arrays.asList("id", "tables", "job_item_count", "active", "create_time", "stop_time");
     }
     
-    @Override
-    public boolean next() {
-        return data.hasNext();
-    }
-    
-    @Override
-    public Collection<Object> getRowData() {
-        return data.next();
-    }
-    
     @Override
     public String getType() {
         return ShowMigrationListStatement.class.getName();
diff --git a/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet b/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
similarity index 68%
copy from kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
copy to kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
index 23ac487d8a4..ad4abdc5252 100644
--- a/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
+++ b/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor
@@ -15,8 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationCheckStatusResultSet
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationListResultSet
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationJobStatusResultSet
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationCheckAlgorithmsResultSet
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationSourceStorageUnitsResultSet
+org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationListExecutor
diff --git a/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet b/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
index 23ac487d8a4..30f68c6ace0 100644
--- a/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
+++ b/kernel/data-pipeline/distsql/handler/src/main/resources/META-INF/services/org.apache.shardingsphere.distsql.handler.resultset.DistSQLResultSet
@@ -16,7 +16,6 @@
 #
 
 org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationCheckStatusResultSet
-org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationListResultSet
 org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationJobStatusResultSet
 org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationCheckAlgorithmsResultSet
 org.apache.shardingsphere.migration.distsql.handler.query.ShowMigrationSourceStorageUnitsResultSet
diff --git a/kernel/data-pipeline/distsql/handler/src/test/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutorTest.java b/kernel/data-pipeline/distsql/handler/src/test/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutorTest.java
new file mode 100644
index 00000000000..ef72b1a9111
--- /dev/null
+++ b/kernel/data-pipeline/distsql/handler/src/test/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationListExecutorTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.migration.distsql.handler.query;
+
+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 ShowMigrationListExecutorTest {
+    
+    @Test
+    public void assertGetColumnNames() {
+        ShowMigrationListExecutor executor = new ShowMigrationListExecutor();
+        Collection<String> columns = executor.getColumnNames();
+        assertThat(columns.size(), is(6));
+        Iterator<String> iterator = columns.iterator();
+        assertThat(iterator.next(), is("id"));
+        assertThat(iterator.next(), is("tables"));
+        assertThat(iterator.next(), is("job_item_count"));
+        assertThat(iterator.next(), is("active"));
+        assertThat(iterator.next(), is("create_time"));
+        assertThat(iterator.next(), is("stop_time"));
+    }
+}