You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by "azexcy (via GitHub)" <gi...@apache.org> on 2023/03/01 08:48:21 UTC

[GitHub] [shardingsphere] azexcy opened a new pull request, #24404: Add show streaming list/status DistSQL for CDC

azexcy opened a new pull request, #24404:
URL: https://github.com/apache/shardingsphere/pull/24404

   Related #22500 
   
   Changes proposed in this pull request:
     - Add show streaming list/status DistSQL for CDC
     - Add unit test
   
   ---
   
   Before committing this PR, I'm sure that I have checked the following options:
   - [ ] My code follows the [code of conduct](https://shardingsphere.apache.org/community/en/involved/conduct/code/) of this project.
   - [ ] I have self-reviewed the commit code.
   - [ ] I have (or in comment I request) added corresponding labels for the pull request.
   - [ ] I have passed maven check locally : `./mvnw clean install -B -T1C -Dmaven.javadoc.skip -Dmaven.jacoco.skip -e`.
   - [ ] I have made corresponding changes to the documentation.
   - [ ] I have added corresponding unit tests for my changes.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz merged pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "sandynz (via GitHub)" <gi...@apache.org>.
sandynz merged PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "sandynz (via GitHub)" <gi...@apache.org>.
sandynz commented on code in PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#discussion_r1122568172


##########
kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/pojo/TableBasedPipelineJobInfo.java:
##########
@@ -32,4 +32,10 @@ public final class TableBasedPipelineJobInfo implements PipelineJobInfo {
     private final String databaseName;
     
     private final String table;
+    
+    public TableBasedPipelineJobInfo(final PipelineJobMetaData jobMetaData, final String table) {
+        this.jobMetaData = jobMetaData;
+        this.table = table;
+        this.databaseName = null;

Review Comment:
   1, `this.databaseName` could be `databaseName`.
   
   2, Should `databaseName = null;` put before `this.table = table;`?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "sandynz (via GitHub)" <gi...@apache.org>.
sandynz commented on code in PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#discussion_r1122556797


##########
kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/pojo/TableBasedPipelineJobInfo.java:
##########
@@ -29,5 +29,7 @@ public final class TableBasedPipelineJobInfo implements PipelineJobInfo {
     
     private final PipelineJobMetaData jobMetaData;
     
+    private final String databaseName;
+    

Review Comment:
   It's better to add a constructor to compatible with other invocations



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "sandynz (via GitHub)" <gi...@apache.org>.
sandynz commented on code in PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#discussion_r1121388616


##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/impl/CDCJobAPI.java:
##########
@@ -286,7 +288,12 @@ protected YamlPipelineJobConfiguration swapToYamlJobConfiguration(final Pipeline
     
     @Override
     protected PipelineJobInfo getJobInfo(final String jobId) {
-        throw new UnsupportedOperationException();
+        JobConfigurationPOJO jobConfigPOJO = getElasticJobConfigPOJO(jobId);
+        PipelineJobMetaData jobMetaData = new PipelineJobMetaData(jobId, !jobConfigPOJO.isDisabled(),
+                jobConfigPOJO.getShardingTotalCount(), jobConfigPOJO.getProps().getProperty("create_time"), jobConfigPOJO.getProps().getProperty("stop_time"), jobConfigPOJO.getJobParameter());
+        CDCJobConfiguration jobConfig = getJobConfiguration(jobConfigPOJO);
+        List<String> tableNames = jobConfig.getSchemaTableNames().stream().map(each -> String.join(".", jobConfig.getDatabase(), each)).collect(Collectors.toList());

Review Comment:
   logic database name could be put in another column, since there's only one database name per job, there might be too much duplicated text when there're much tables.



##########
test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/ral/impl/cdc/ShowStreamingStatusStatementAssert.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.test.it.sql.parser.internal.asserts.statement.ral.impl.cdc;
+
+import org.apache.shardingsphere.cdc.distsql.statement.ShowStreamingStatusStatement;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.it.sql.parser.internal.asserts.statement.ral.impl.migration.JobIdAssert;
+import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.jaxb.statement.ral.cdc.ShowStreamingStatusStatementTestCase;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Show streaming status statement assert.
+ */
+public final class ShowStreamingStatusStatementAssert {
+    
+    /**
+     * Assert show migration status statement is correct with expected parser result.
+     *
+     * @param assertContext assert context
+     * @param actual actual show migration status statement
+     * @param expected expected show migration status statement test case

Review Comment:
   `migration` words should be `CDC`



##########
kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/cdc/distsql/handler/query/ShowStreamingJobStatusExecutor.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.cdc.distsql.handler.query;
+
+import org.apache.shardingsphere.cdc.distsql.statement.ShowStreamingStatusStatement;
+import org.apache.shardingsphere.data.pipeline.api.job.progress.InventoryIncrementalJobItemProgress;
+import org.apache.shardingsphere.data.pipeline.api.pojo.InventoryIncrementalJobItemInfo;
+import org.apache.shardingsphere.data.pipeline.core.api.InventoryIncrementalJobAPI;
+import org.apache.shardingsphere.data.pipeline.core.api.PipelineJobAPI;
+import org.apache.shardingsphere.distsql.handler.ral.query.QueryableRALExecutor;
+import org.apache.shardingsphere.infra.merge.result.impl.local.LocalDataQueryResultRow;
+import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPILoader;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Show streaming job status executor.
+ */
+public final class ShowStreamingJobStatusExecutor implements QueryableRALExecutor<ShowStreamingStatusStatement> {
+    
+    @Override
+    public Collection<LocalDataQueryResultRow> getRows(final ShowStreamingStatusStatement sqlStatement) {
+        InventoryIncrementalJobAPI jobAPI = (InventoryIncrementalJobAPI) TypedSPILoader.getService(PipelineJobAPI.class, "CDC");
+        List<InventoryIncrementalJobItemInfo> jobItemInfos = jobAPI.getJobItemInfos(sqlStatement.getJobId());
+        long currentTimeMillis = System.currentTimeMillis();
+        return jobItemInfos.stream().map(each -> {
+            LocalDataQueryResultRow row;
+            InventoryIncrementalJobItemProgress jobItemProgress = each.getJobItemProgress();
+            if (null != jobItemProgress) {
+                String incrementalIdleSeconds = "";
+                if (jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis() > 0) {
+                    long latestActiveTimeMillis = Math.max(each.getStartTimeMillis(), jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis());
+                    incrementalIdleSeconds = String.valueOf(TimeUnit.MILLISECONDS.toSeconds(currentTimeMillis - latestActiveTimeMillis));
+                }
+                row = new LocalDataQueryResultRow(each.getShardingItem(), jobItemProgress.getDataSourceName(), jobItemProgress.getStatus(),
+                        jobItemProgress.isActive() ? Boolean.TRUE.toString() : Boolean.FALSE.toString(), jobItemProgress.getProcessedRecordsCount(), each.getInventoryFinishedPercentage(),
+                        incrementalIdleSeconds, each.getErrorMessage());
+            } else {
+                row = new LocalDataQueryResultRow(each.getShardingItem(), "", "", "", "", "", "", each.getErrorMessage());
+            }
+            return row;

Review Comment:
   1, Could we extract this code block in new method?
   
   2, It's better not put too much code in `if` block



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] sandynz commented on a diff in pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "sandynz (via GitHub)" <gi...@apache.org>.
sandynz commented on code in PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#discussion_r1122523146


##########
kernel/data-pipeline/distsql/handler/src/main/java/org/apache/shardingsphere/migration/distsql/handler/query/ShowMigrationJobStatusExecutor.java:
##########
@@ -42,23 +42,23 @@ public Collection<LocalDataQueryResultRow> getRows(final ShowMigrationStatusStat
         InventoryIncrementalJobAPI jobAPI = (InventoryIncrementalJobAPI) TypedSPILoader.getService(PipelineJobAPI.class, "MIGRATION");
         List<InventoryIncrementalJobItemInfo> jobItemInfos = jobAPI.getJobItemInfos(sqlStatement.getJobId());
         long currentTimeMillis = System.currentTimeMillis();
-        return jobItemInfos.stream().map(each -> {
-            LocalDataQueryResultRow row;
-            InventoryIncrementalJobItemProgress jobItemProgress = each.getJobItemProgress();
-            if (null != jobItemProgress) {
-                String incrementalIdleSeconds = "";
-                if (jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis() > 0) {
-                    long latestActiveTimeMillis = Math.max(each.getStartTimeMillis(), jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis());
-                    incrementalIdleSeconds = String.valueOf(TimeUnit.MILLISECONDS.toSeconds(currentTimeMillis - latestActiveTimeMillis));
-                }
-                row = new LocalDataQueryResultRow(each.getShardingItem(), jobItemProgress.getDataSourceName(), jobItemProgress.getStatus(),
-                        jobItemProgress.isActive() ? Boolean.TRUE.toString() : Boolean.FALSE.toString(), jobItemProgress.getProcessedRecordsCount(), each.getInventoryFinishedPercentage(),
-                        incrementalIdleSeconds, each.getErrorMessage());
-            } else {
-                row = new LocalDataQueryResultRow(each.getShardingItem(), "", "", "", "", "", "", each.getErrorMessage());
+        return jobItemInfos.stream().map(each -> generateResultRow(each, currentTimeMillis)).collect(Collectors.toList());
+    }
+    
+    private LocalDataQueryResultRow generateResultRow(final InventoryIncrementalJobItemInfo jobItemInfo, final long currentTimeMillis) {
+        InventoryIncrementalJobItemProgress jobItemProgress = jobItemInfo.getJobItemProgress();
+        if (null != jobItemProgress) {
+            String incrementalIdleSeconds = "";
+            if (jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis() > 0) {
+                long latestActiveTimeMillis = Math.max(jobItemInfo.getStartTimeMillis(), jobItemProgress.getIncremental().getIncrementalLatestActiveTimeMillis());
+                incrementalIdleSeconds = String.valueOf(TimeUnit.MILLISECONDS.toSeconds(currentTimeMillis - latestActiveTimeMillis));
             }
-            return row;
-        }).collect(Collectors.toList());
+            return new LocalDataQueryResultRow(jobItemInfo.getShardingItem(), jobItemProgress.getDataSourceName(), jobItemProgress.getStatus(),
+                    jobItemProgress.isActive() ? Boolean.TRUE.toString() : Boolean.FALSE.toString(), jobItemProgress.getProcessedRecordsCount(), jobItemInfo.getInventoryFinishedPercentage(),
+                    incrementalIdleSeconds, jobItemInfo.getErrorMessage());
+        } else {
+            return new LocalDataQueryResultRow(jobItemInfo.getShardingItem(), "", "", "", "", "", "", jobItemInfo.getErrorMessage());
+        }

Review Comment:
   The `else` block could be extracted before `if` block and `return`, and remove `if` block.
   
   From [Code of Conduct]( https://shardingsphere.apache.org/community/en/involved/conduct/code/ ):
   > Use guard clauses in priority.
   



##########
kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/pojo/CDCTableBasedPipelineJobInfo.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.data.pipeline.api.pojo;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+/**
+ * CDC table based pipeline job info.
+ */
+@Getter
+@RequiredArgsConstructor
+public class CDCTableBasedPipelineJobInfo implements PipelineJobInfo {
+    
+    private final PipelineJobMetaData jobMetaData;
+    
+    private final String databaseName;
+    
+    private final String schemaTableNames;

Review Comment:
   1, Could we add `databaseName` in `TableBasedPipelineJobInfo`? It's the common impl
   
   2, Could we just reuse `table` in `TableBasedPipelineJobInfo`, but not add `schemaTableNames`?
   



##########
kernel/data-pipeline/cdc/core/src/main/java/org/apache/shardingsphere/data/pipeline/cdc/api/pojo/StreamDataParameter.java:
##########
@@ -31,6 +31,7 @@
 @Getter
 public final class StreamDataParameter {
     
+    // TODO rename to databaseName
     private final String database;

Review Comment:
   It could be renamed now, just keep `database` in protobuf protocol definition for now



##########
test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/ral/impl/cdc/ShowStreamingStatusStatementAssert.java:
##########
@@ -31,7 +31,7 @@
 public final class ShowStreamingStatusStatementAssert {
     
     /**
-     * Assert show migration status statement is correct with expected parser result.
+     * Assert show streaming status statement is correct with expected parser result.
      *
      * @param assertContext assert context
      * @param actual actual show migration status statement

Review Comment:
   There still has `migration`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] codecov-commenter commented on pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#issuecomment-1451282094

   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#24404](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (b7cc812) into [master](https://codecov.io/gh/apache/shardingsphere/commit/1a1e5060f04516c5d1e887aa32ac5b8b294246ab?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1a1e506) will **decrease** coverage by `0.04%`.
   > The diff coverage is `1.44%`.
   
   > :exclamation: Current head b7cc812 differs from pull request most recent head 6a31802. Consider uploading reports for the commit 6a31802 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #24404      +/-   ##
   ============================================
   - Coverage     50.01%   49.97%   -0.04%     
     Complexity     1583     1583              
   ============================================
     Files          3248     3254       +6     
     Lines         53524    53565      +41     
     Branches       9862     9868       +6     
   ============================================
     Hits          26770    26770              
   - Misses        24362    24403      +41     
     Partials       2392     2392              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/pipeline/api/pojo/TableBasedPipelineJobInfo.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvYXBpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9kYXRhL3BpcGVsaW5lL2FwaS9wb2pvL1RhYmxlQmFzZWRQaXBlbGluZUpvYkluZm8uamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ngsphere/data/pipeline/cdc/api/impl/CDCJobAPI.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvY2RjL2NvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RhdGEvcGlwZWxpbmUvY2RjL2FwaS9pbXBsL0NEQ0pvYkFQSS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...a/pipeline/cdc/config/job/CDCJobConfiguration.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvY2RjL2NvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RhdGEvcGlwZWxpbmUvY2RjL2NvbmZpZy9qb2IvQ0RDSm9iQ29uZmlndXJhdGlvbi5qYXZh) | `0.00% <ø> (ø)` | |
   | [...pipeline/cdc/yaml/job/YamlCDCJobConfiguration.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvY2RjL2NvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2RhdGEvcGlwZWxpbmUvY2RjL3lhbWwvam9iL1lhbWxDRENKb2JDb25maWd1cmF0aW9uLmphdmE=) | `75.00% <ø> (ø)` | |
   | [.../handler/query/ShowStreamingJobStatusExecutor.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9oYW5kbGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9jZGMvZGlzdHNxbC9oYW5kbGVyL3F1ZXJ5L1Nob3dTdHJlYW1pbmdKb2JTdGF0dXNFeGVjdXRvci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...stsql/handler/query/ShowStreamingListExecutor.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9oYW5kbGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9jZGMvZGlzdHNxbC9oYW5kbGVyL3F1ZXJ5L1Nob3dTdHJlYW1pbmdMaXN0RXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../handler/query/ShowMigrationJobStatusExecutor.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9oYW5kbGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9taWdyYXRpb24vZGlzdHNxbC9oYW5kbGVyL3F1ZXJ5L1Nob3dNaWdyYXRpb25Kb2JTdGF0dXNFeGVjdXRvci5qYXZh) | `11.76% <0.00%> (+1.76%)` | :arrow_up: |
   | [...here/cdc/distsql/parser/core/CDCDistSQLParser.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9wYXJzZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2NkYy9kaXN0c3FsL3BhcnNlci9jb3JlL0NEQ0Rpc3RTUUxQYXJzZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...istsql/parser/core/CDCDistSQLStatementVisitor.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9wYXJzZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2NkYy9kaXN0c3FsL3BhcnNlci9jb3JlL0NEQ0Rpc3RTUUxTdGF0ZW1lbnRWaXNpdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...parser/facade/CDCDistSQLStatementParserFacade.java](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a2VybmVsL2RhdGEtcGlwZWxpbmUvZGlzdHNxbC9wYXJzZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2NkYy9kaXN0c3FsL3BhcnNlci9mYWNhZGUvQ0RDRGlzdFNRTFN0YXRlbWVudFBhcnNlckZhY2FkZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [3 more](https://codecov.io/gh/apache/shardingsphere/pull/24404?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [shardingsphere] azexcy commented on pull request #24404: Add show streaming list/status DistSQL for CDC

Posted by "azexcy (via GitHub)" <gi...@apache.org>.
azexcy commented on PR #24404:
URL: https://github.com/apache/shardingsphere/pull/24404#issuecomment-1449610769

   example
   
   ```
   mysql> show streaming list;
   +---------------------------------------+---------------------+----------------+--------+---------------------+-----------+
   | id                                    | tables              | job_item_count | active | create_time         | stop_time |
   +---------------------------------------+---------------------+----------------+--------+---------------------+-----------+
   | j03016a67bc12943b8e602e9f880b46263dc7 | sharding_db.t_order | 2              | true   | 2023-03-01 15:54:01 | NULL      |
   +---------------------------------------+---------------------+----------------+--------+---------------------+-----------+
   1 row in set (0.04 sec)
   
   mysql> show streaming status 'j03016a67bc12943b8e602e9f880b46263dc7';
   +------+-------------+--------------------------+--------+-------------------------+-------------------------------+--------------------------+---------------+
   | item | data_source | status                   | active | processed_records_count | inventory_finished_percentage | incremental_idle_seconds | error_message |
   +------+-------------+--------------------------+--------+-------------------------+-------------------------------+--------------------------+---------------+
   | 0    | ds_0        | EXECUTE_INCREMENTAL_TASK | true   | 0                       | 100                           |                          |               |
   | 1    | ds_1        | EXECUTE_INCREMENTAL_TASK | true   | 1                       | 100                           | 3612                     |               |
   +------+-------------+--------------------------+--------+-------------------------+-------------------------------+--------------------------+---------------+
   2 rows in set (0.53 sec)
   
   mysql>
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org