You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/11/05 12:38:59 UTC

[shardingsphere] branch master updated: Cleanup pipeline exception (#21974)

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

duanzhengqiang 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 a79c488aa35 Cleanup pipeline exception (#21974)
a79c488aa35 is described below

commit a79c488aa35b3c1e0620152aa5a203e0c923eba3
Author: Hongsheng Zhong <zh...@apache.org>
AuthorDate: Sat Nov 5 20:38:46 2022 +0800

    Cleanup pipeline exception (#21974)
---
 .../user-manual/error-code/sql-error-code.cn.md    |  4 +--
 .../user-manual/error-code/sql-error-code.en.md    |  4 +--
 ...a => ConsistencyCheckJobNotFoundException.java} |  9 +++---
 .../job/PipelineJobExecutionException.java         | 34 ----------------------
 .../job/PipelineJobNotFoundException.java          |  1 -
 .../CreateExistsProcessConfigurationException.java | 34 ----------------------
 .../ConsistencyCheckJobAPIImpl.java                |  8 ++---
 7 files changed, 10 insertions(+), 84 deletions(-)

diff --git a/docs/document/content/user-manual/error-code/sql-error-code.cn.md b/docs/document/content/user-manual/error-code/sql-error-code.cn.md
index 2dc72e49e4c..04f97b0e588 100644
--- a/docs/document/content/user-manual/error-code/sql-error-code.cn.md
+++ b/docs/document/content/user-manual/error-code/sql-error-code.cn.md
@@ -91,7 +91,6 @@ SQL 错误码以标准的 SQL State,Vendor Code 和详细错误信息提供,
 
 | SQL State | Vendor Code | 错误信息 |
 | --------- | ----------- | ------ |
-| 44000     | 18001       | Created process configuration already existed. |
 | 44000     | 18002       | Altered process configuration does not exist. |
 | HY000     | 18020       | Failed to get DDL for table \`%s\`. |
 | 42S01     | 18030       | Duplicate resource names \`%s\`. |
@@ -115,8 +114,7 @@ SQL 错误码以标准的 SQL State,Vendor Code 和详细错误信息提供,
 | HY000     | 18091       | Importer job write data failed. |
 | 08000     | 18092       | Get binlog position failed by job \`%s\`, reason is: %s |
 | HY000     | 18093       | Can not poll event because of binlog sync channel already closed. |
-| HY000     | 18094       | Task \`%s\` execute failed. |
-| HY000     | 18095       | Job has already finished, please run \`CHECK MIGRATION %s\` to start a new data consistency check job. |
+| HY000     | 18095       | Can not find consistency check job of \`%s\`. |
 | HY000     | 18096       | Uncompleted consistency check job \`%s\` exists. |
 
 ### DistSQL
diff --git a/docs/document/content/user-manual/error-code/sql-error-code.en.md b/docs/document/content/user-manual/error-code/sql-error-code.en.md
index 51efcb8ae1b..fdda7dafdbb 100644
--- a/docs/document/content/user-manual/error-code/sql-error-code.en.md
+++ b/docs/document/content/user-manual/error-code/sql-error-code.en.md
@@ -91,7 +91,6 @@ SQL error codes provide by standard `SQL State`, `Vendor Code` and `Reason`, whi
 
 | SQL State | Vendor Code | Reason |
 | --------- | ----------- | ------ |
-| 44000     | 18001       | Created process configuration already existed. |
 | 44000     | 18002       | Altered process configuration does not exist. |
 | HY000     | 18020       | Failed to get DDL for table \`%s\`. |
 | 42S01     | 18030       | Duplicate resource names \`%s\`. |
@@ -115,8 +114,7 @@ SQL error codes provide by standard `SQL State`, `Vendor Code` and `Reason`, whi
 | HY000     | 18091       | Importer job write data failed. |
 | 08000     | 18092       | Get binlog position failed by job \`%s\`, reason is: %s |
 | HY000     | 18093       | Can not poll event because of binlog sync channel already closed. |
-| HY000     | 18094       | Task \`%s\` execute failed. |
-| HY000     | 18095       | Job has already finished, please run \`CHECK MIGRATION %s\` to start a new data consistency check job. |
+| HY000     | 18095       | Can not find consistency check job of \`%s\`. |
 | HY000     | 18096       | Uncompleted consistency check job \`%s\` exists. |
 
 ### DistSQL
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobHasAlreadyFinishedException.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/ConsistencyCheckJobNotFoundException.java
similarity index 77%
rename from kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobHasAlreadyFinishedException.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/ConsistencyCheckJobNotFoundException.java
index c8853ff732e..e49b458ecc9 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobHasAlreadyFinishedException.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/ConsistencyCheckJobNotFoundException.java
@@ -21,14 +21,13 @@ import org.apache.shardingsphere.data.pipeline.core.exception.PipelineSQLExcepti
 import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
 
 /**
- * Pipeline job has already finished exception.
+ * Can not find consistency check job exception.
  */
-// TODO keep it for now, it might be used later
-public final class PipelineJobHasAlreadyFinishedException extends PipelineSQLException {
+public final class ConsistencyCheckJobNotFoundException extends PipelineSQLException {
     
     private static final long serialVersionUID = 6881217592831423520L;
     
-    public PipelineJobHasAlreadyFinishedException() {
-        super(XOpenSQLState.GENERAL_ERROR, 95, "Data consistency check job has already finished.");
+    public ConsistencyCheckJobNotFoundException(final String jobId) {
+        super(XOpenSQLState.GENERAL_ERROR, 95, String.format("Can not find consistency check job of `%s`.", jobId));
     }
 }
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobExecutionException.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobExecutionException.java
deleted file mode 100644
index cd1c8c831e0..00000000000
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobExecutionException.java
+++ /dev/null
@@ -1,34 +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.
- */
-
-package org.apache.shardingsphere.data.pipeline.core.exception.job;
-
-import org.apache.shardingsphere.data.pipeline.core.exception.PipelineSQLException;
-import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
-
-/**
- * Pipeline job execution exception.
- */
-// TODO delete, not used
-public final class PipelineJobExecutionException extends PipelineSQLException {
-    
-    private static final long serialVersionUID = -8462847591661221914L;
-    
-    public PipelineJobExecutionException(final String taskId, final Exception cause) {
-        super(XOpenSQLState.GENERAL_ERROR, 94, String.format("Task `%s` execute failed.", taskId), cause);
-    }
-}
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobNotFoundException.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobNotFoundException.java
index 34e0e5f421c..f759e0ec0cf 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobNotFoundException.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/job/PipelineJobNotFoundException.java
@@ -27,7 +27,6 @@ public final class PipelineJobNotFoundException extends PipelineSQLException {
     
     private static final long serialVersionUID = -903289953649758722L;
     
-    // TODO it's misused sometimes
     public PipelineJobNotFoundException(final String jobId) {
         super(XOpenSQLState.GENERAL_ERROR, 80, String.format("Can not find pipeline job `%s`.", jobId));
     }
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/metadata/CreateExistsProcessConfigurationException.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/metadata/CreateExistsProcessConfigurationException.java
deleted file mode 100644
index 24da42badc9..00000000000
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/exception/metadata/CreateExistsProcessConfigurationException.java
+++ /dev/null
@@ -1,34 +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.
- */
-
-package org.apache.shardingsphere.data.pipeline.core.exception.metadata;
-
-import org.apache.shardingsphere.data.pipeline.core.exception.PipelineSQLException;
-import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
-
-/**
- * Create exists process configuration exception.
- */
-// TODO delete, not used
-public final class CreateExistsProcessConfigurationException extends PipelineSQLException {
-    
-    private static final long serialVersionUID = 8134085267509955091L;
-    
-    public CreateExistsProcessConfigurationException() {
-        super(XOpenSQLState.CHECK_OPTION_VIOLATION, 1, "Created process configuration already existed.");
-    }
-}
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/ConsistencyCheckJobAPIImpl.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/ConsistencyCheckJobAPIImpl.java
index c2b4ba02466..4f4f66f74e3 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/ConsistencyCheckJobAPIImpl.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/ConsistencyCheckJobAPIImpl.java
@@ -39,7 +39,7 @@ import org.apache.shardingsphere.data.pipeline.core.api.GovernanceRepositoryAPI;
 import org.apache.shardingsphere.data.pipeline.core.api.InventoryIncrementalJobAPI;
 import org.apache.shardingsphere.data.pipeline.core.api.PipelineAPIFactory;
 import org.apache.shardingsphere.data.pipeline.core.api.impl.AbstractPipelineJobAPIImpl;
-import org.apache.shardingsphere.data.pipeline.core.exception.job.PipelineJobNotFoundException;
+import org.apache.shardingsphere.data.pipeline.core.exception.job.ConsistencyCheckJobNotFoundException;
 import org.apache.shardingsphere.data.pipeline.core.exception.job.UncompletedConsistencyCheckJobExistsException;
 import org.apache.shardingsphere.data.pipeline.core.job.PipelineJobIdUtils;
 import org.apache.shardingsphere.data.pipeline.core.job.progress.yaml.YamlConsistencyCheckJobProgress;
@@ -156,7 +156,7 @@ public final class ConsistencyCheckJobAPIImpl extends AbstractPipelineJobAPIImpl
     public void startByParentJobId(final String parentJobId) {
         log.info("Start check job by parent job id: {}", parentJobId);
         Optional<String> checkLatestJobId = PipelineAPIFactory.getGovernanceRepositoryAPI().getCheckLatestJobId(parentJobId);
-        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new PipelineJobNotFoundException(parentJobId));
+        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new ConsistencyCheckJobNotFoundException(parentJobId));
         startDisabledJob(checkLatestJobId.get());
     }
     
@@ -164,14 +164,14 @@ public final class ConsistencyCheckJobAPIImpl extends AbstractPipelineJobAPIImpl
     public void stopByParentJobId(final String parentJobId) {
         log.info("Stop check job by parent job id: {}", parentJobId);
         Optional<String> checkLatestJobId = PipelineAPIFactory.getGovernanceRepositoryAPI().getCheckLatestJobId(parentJobId);
-        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new PipelineJobNotFoundException(parentJobId));
+        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new ConsistencyCheckJobNotFoundException(parentJobId));
         stop(checkLatestJobId.get());
     }
     
     @Override
     public ConsistencyCheckJobProgressInfo getJobProgressInfo(final String parentJobId) {
         Optional<String> checkLatestJobId = PipelineAPIFactory.getGovernanceRepositoryAPI().getCheckLatestJobId(parentJobId);
-        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new PipelineJobNotFoundException(parentJobId));
+        ShardingSpherePreconditions.checkState(checkLatestJobId.isPresent(), () -> new ConsistencyCheckJobNotFoundException(parentJobId));
         String checkJobId = checkLatestJobId.get();
         ConsistencyCheckJobProgress jobItemProgress = getJobItemProgress(checkJobId, 0);
         ConsistencyCheckJobProgressInfo result = new ConsistencyCheckJobProgressInfo();