You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by az...@apache.org on 2023/03/09 13:14:03 UTC

[shardingsphere] branch master updated: Extract AbstractChangedJobConfigurationProcessor for common usage (#24527)

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

azexin 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 b3c705b7796 Extract AbstractChangedJobConfigurationProcessor for common usage (#24527)
b3c705b7796 is described below

commit b3c705b7796082ad42e1b55a389fbff2de660720
Author: Hongsheng Zhong <zh...@apache.org>
AuthorDate: Thu Mar 9 21:13:45 2023 +0800

    Extract AbstractChangedJobConfigurationProcessor for common usage (#24527)
    
    * Rename PipelineChangedJobConfigurationProcessor to ChangedJobConfigurationProcessor
    
    * Rename ChangedJobConfigurationDispatcher to ConfigMetaDataChangedEventHandler
    
    * Move ChangedJobConfigurationProcessor
    
    * Rename ChangedJobConfigurationProcessor sub-classes
    
    * Extract AbstractChangedJobConfigurationProcessor for common usage
---
 .../data/pipeline/api/job/PipelineJob.java         |  3 +-
 .../data/pipeline/api/job/PipelineJobId.java       |  0
 .../ChangedJobConfigurationProcessor.java}         |  6 +-
 .../AbstractChangedJobConfigurationProcessor.java} | 43 +++++++-----
 ...java => ConfigMetaDataChangedEventHandler.java} |  8 +--
 ...ent.handler.PipelineMetaDataChangedEventHandler |  2 +-
 ...dConsistencyCheckJobConfigurationProcessor.java | 47 +++++++++++++
 ...tencyCheckChangedJobConfigurationProcessor.java | 78 ----------------------
 ...fig.processor.ChangedJobConfigurationProcessor} |  2 +-
 .../ChangedMigrationJobConfigurationProcessor.java | 50 ++++++++++++++
 ...fig.processor.ChangedJobConfigurationProcessor} |  2 +-
 11 files changed, 135 insertions(+), 106 deletions(-)

diff --git a/kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java
similarity index 92%
rename from kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java
index d794fc58853..996c06e9b15 100644
--- a/kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJob.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.data.pipeline.api.job;
 
 import org.apache.shardingsphere.data.pipeline.api.task.PipelineTasksRunner;
+import org.apache.shardingsphere.elasticjob.api.ElasticJob;
 
 import java.util.Collection;
 import java.util.Optional;
@@ -25,7 +26,7 @@ import java.util.Optional;
 /**
  * Pipeline job.
  */
-public interface PipelineJob {
+public interface PipelineJob extends ElasticJob {
     
     /**
      * Get tasks runner.
diff --git a/kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJobId.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJobId.java
similarity index 100%
rename from kernel/data-pipeline/api/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJobId.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/api/job/PipelineJobId.java
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/PipelineChangedJobConfigurationProcessor.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/ChangedJobConfigurationProcessor.java
similarity index 90%
rename from kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/PipelineChangedJobConfigurationProcessor.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/ChangedJobConfigurationProcessor.java
index 608f64143d6..d0ed6592fe3 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/PipelineChangedJobConfigurationProcessor.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/ChangedJobConfigurationProcessor.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler;
+package org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor;
 
 import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
 import org.apache.shardingsphere.infra.util.spi.type.typed.TypedSPI;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent.Type;
 
 /**
- * Pipeline changed job configuration processor.
+ * Changed job configuration processor.
  */
-public interface PipelineChangedJobConfigurationProcessor extends TypedSPI {
+public interface ChangedJobConfigurationProcessor extends TypedSPI {
     
     /**
      * Process changed job configuration.
diff --git a/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/MigrationChangedJobConfigurationProcessor.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/impl/AbstractChangedJobConfigurationProcessor.java
similarity index 67%
rename from kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/MigrationChangedJobConfigurationProcessor.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/impl/AbstractChangedJobConfigurationProcessor.java
index e12df4352cc..9abebfbfdde 100644
--- a/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/MigrationChangedJobConfigurationProcessor.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/config/processor/impl/AbstractChangedJobConfigurationProcessor.java
@@ -15,41 +15,37 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.scenario.migration.metadata.processor;
+package org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.impl;
 
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.core.api.PipelineAPIFactory;
+import org.apache.shardingsphere.data.pipeline.core.job.AbstractPipelineJob;
 import org.apache.shardingsphere.data.pipeline.core.job.PipelineJobCenter;
 import org.apache.shardingsphere.data.pipeline.core.metadata.node.PipelineMetaDataNode;
-import org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor;
+import org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor;
 import org.apache.shardingsphere.data.pipeline.core.util.PipelineDistributedBarrier;
-import org.apache.shardingsphere.data.pipeline.scenario.migration.MigrationJob;
-import org.apache.shardingsphere.data.pipeline.scenario.migration.MigrationJobType;
-import org.apache.shardingsphere.data.pipeline.scenario.migration.prepare.MigrationJobPreparer;
-import org.apache.shardingsphere.data.pipeline.yaml.job.YamlMigrationJobConfigurationSwapper;
+import org.apache.shardingsphere.data.pipeline.spi.job.JobType;
 import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
 import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.OneOffJobBootstrap;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent.Type;
 
 /**
- * Migration job configuration changed processor.
+ * Abstract changed job configuration processor.
  */
 @Slf4j
-public final class MigrationChangedJobConfigurationProcessor implements PipelineChangedJobConfigurationProcessor {
+public abstract class AbstractChangedJobConfigurationProcessor implements ChangedJobConfigurationProcessor {
     
     @Override
     public void process(final Type eventType, final JobConfiguration jobConfig) {
-        String jobId = jobConfig.getJobName();
         boolean disabled = jobConfig.isDisabled();
         if (disabled) {
-            for (Integer each : PipelineJobCenter.getShardingItems(jobId)) {
-                PipelineDistributedBarrier.getInstance().persistEphemeralChildrenNode(PipelineMetaDataNode.getJobBarrierDisablePath(jobId), each);
-            }
+            onDisabled(jobConfig);
         }
         boolean deleted = Type.DELETED == eventType;
         if (deleted) {
-            new MigrationJobPreparer().cleanup(new YamlMigrationJobConfigurationSwapper().swapToObject(jobConfig.getJobParameter()));
+            onDeleted(jobConfig);
         }
+        String jobId = jobConfig.getJobName();
         if (disabled || deleted) {
             PipelineJobCenter.stop(jobId);
             return;
@@ -60,7 +56,7 @@ public final class MigrationChangedJobConfigurationProcessor implements Pipeline
                 if (PipelineJobCenter.isJobExisting(jobId)) {
                     log.info("{} added to executing jobs failed since it already exists", jobId);
                 } else {
-                    execute(jobConfig);
+                    executeJob(jobConfig);
                 }
                 break;
             default:
@@ -68,16 +64,29 @@ public final class MigrationChangedJobConfigurationProcessor implements Pipeline
         }
     }
     
-    private void execute(final JobConfiguration jobConfig) {
-        MigrationJob job = new MigrationJob();
+    protected void onDisabled(final JobConfiguration jobConfig) {
+        String jobId = jobConfig.getJobName();
+        for (Integer each : PipelineJobCenter.getShardingItems(jobId)) {
+            PipelineDistributedBarrier.getInstance().persistEphemeralChildrenNode(PipelineMetaDataNode.getJobBarrierDisablePath(jobId), each);
+        }
+    }
+    
+    protected abstract void onDeleted(JobConfiguration jobConfig);
+    
+    protected void executeJob(final JobConfiguration jobConfig) {
+        AbstractPipelineJob job = buildPipelineJob();
         PipelineJobCenter.addJob(jobConfig.getJobName(), job);
         OneOffJobBootstrap oneOffJobBootstrap = new OneOffJobBootstrap(PipelineAPIFactory.getRegistryCenter(), job, jobConfig);
         job.setJobBootstrap(oneOffJobBootstrap);
         oneOffJobBootstrap.execute();
     }
     
+    protected abstract AbstractPipelineJob buildPipelineJob();
+    
+    protected abstract JobType getJobType();
+    
     @Override
     public String getType() {
-        return new MigrationJobType().getTypeName();
+        return getJobType().getTypeName();
     }
 }
diff --git a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ChangedJobConfigurationDispatcher.java b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ConfigMetaDataChangedEventHandler.java
similarity index 88%
rename from kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ChangedJobConfigurationDispatcher.java
rename to kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ConfigMetaDataChangedEventHandler.java
index 2b3f41a47d7..670cff12971 100644
--- a/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ChangedJobConfigurationDispatcher.java
+++ b/kernel/data-pipeline/core/src/main/java/org/apache/shardingsphere/data/pipeline/core/metadata/node/event/handler/impl/ConfigMetaDataChangedEventHandler.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.core.job.PipelineJobIdUtils;
 import org.apache.shardingsphere.data.pipeline.core.metadata.node.PipelineMetaDataNode;
-import org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor;
+import org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor;
 import org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineMetaDataChangedEventHandler;
 import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
 import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO;
@@ -31,10 +31,10 @@ import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEve
 import java.util.regex.Pattern;
 
 /**
- * Changed job configuration dispatcher.
+ * Config meta data changed event handler.
  */
 @Slf4j
-public final class ChangedJobConfigurationDispatcher implements PipelineMetaDataChangedEventHandler {
+public final class ConfigMetaDataChangedEventHandler implements PipelineMetaDataChangedEventHandler {
     
     @Override
     public Pattern getKeyPattern() {
@@ -53,7 +53,7 @@ public final class ChangedJobConfigurationDispatcher implements PipelineMetaData
             return;
         }
         log.info("{} job configuration: {}, disabled={}", event.getType(), event.getKey(), jobConfig.isDisabled());
-        TypedSPILoader.findService(PipelineChangedJobConfigurationProcessor.class, PipelineJobIdUtils.parseJobType(jobConfig.getJobName()).getTypeName())
+        TypedSPILoader.findService(ChangedJobConfigurationProcessor.class, PipelineJobIdUtils.parseJobType(jobConfig.getJobName()).getTypeName())
                 .ifPresent(optional -> optional.process(event.getType(), jobConfig));
     }
 }
diff --git a/kernel/data-pipeline/core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineMetaDataChangedEventHandler b/kernel/data-pipeline/core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineMetaDataChangedEventHandler
index 56b2c72dd8a..0e239244992 100644
--- a/kernel/data-pipeline/core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineMetaDataChangedEventHandler
+++ b/kernel/data-pipeline/core/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineMetaDataChangedEventHandler
@@ -15,5 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.impl.ChangedJobConfigurationDispatcher
+org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.impl.ConfigMetaDataChangedEventHandler
 org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.impl.BarrierMetaDataChangedEventHandler
diff --git a/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ChangedConsistencyCheckJobConfigurationProcessor.java b/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ChangedConsistencyCheckJobConfigurationProcessor.java
new file mode 100644
index 00000000000..4a1205a4606
--- /dev/null
+++ b/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ChangedConsistencyCheckJobConfigurationProcessor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.scenario.consistencycheck.metadata.processor;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.core.job.AbstractPipelineJob;
+import org.apache.shardingsphere.data.pipeline.core.job.type.ConsistencyCheckJobType;
+import org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.impl.AbstractChangedJobConfigurationProcessor;
+import org.apache.shardingsphere.data.pipeline.scenario.consistencycheck.ConsistencyCheckJob;
+import org.apache.shardingsphere.data.pipeline.spi.job.JobType;
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+
+/**
+ * Changed consistency check job configuration processor.
+ */
+@Slf4j
+public final class ChangedConsistencyCheckJobConfigurationProcessor extends AbstractChangedJobConfigurationProcessor {
+    
+    @Override
+    protected void onDeleted(final JobConfiguration jobConfig) {
+    }
+    
+    @Override
+    protected AbstractPipelineJob buildPipelineJob() {
+        return new ConsistencyCheckJob();
+    }
+    
+    @Override
+    protected JobType getJobType() {
+        return new ConsistencyCheckJobType();
+    }
+}
diff --git a/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ConsistencyCheckChangedJobConfigurationProcessor.java b/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ConsistencyCheckChangedJobConfigurationProcessor.java
deleted file mode 100644
index 84e083c0cd5..00000000000
--- a/kernel/data-pipeline/scenario/consistencycheck/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/consistencycheck/metadata/processor/ConsistencyCheckChangedJobConfigurationProcessor.java
+++ /dev/null
@@ -1,78 +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.scenario.consistencycheck.metadata.processor;
-
-import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.data.pipeline.core.api.PipelineAPIFactory;
-import org.apache.shardingsphere.data.pipeline.core.job.PipelineJobCenter;
-import org.apache.shardingsphere.data.pipeline.core.job.type.ConsistencyCheckJobType;
-import org.apache.shardingsphere.data.pipeline.core.metadata.node.PipelineMetaDataNode;
-import org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor;
-import org.apache.shardingsphere.data.pipeline.core.util.PipelineDistributedBarrier;
-import org.apache.shardingsphere.data.pipeline.scenario.consistencycheck.ConsistencyCheckJob;
-import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
-import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.OneOffJobBootstrap;
-import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent.Type;
-
-/**
- * Consistency check changed job configuration processor.
- */
-@Slf4j
-public final class ConsistencyCheckChangedJobConfigurationProcessor implements PipelineChangedJobConfigurationProcessor {
-    
-    @Override
-    public void process(final Type eventType, final JobConfiguration jobConfig) {
-        String jobId = jobConfig.getJobName();
-        boolean disabled = jobConfig.isDisabled();
-        if (disabled) {
-            for (Integer each : PipelineJobCenter.getShardingItems(jobId)) {
-                PipelineDistributedBarrier.getInstance().persistEphemeralChildrenNode(PipelineMetaDataNode.getJobBarrierDisablePath(jobId), each);
-            }
-        }
-        boolean deleted = Type.DELETED == eventType;
-        if (disabled || deleted) {
-            PipelineJobCenter.stop(jobId);
-            return;
-        }
-        switch (eventType) {
-            case ADDED:
-            case UPDATED:
-                if (PipelineJobCenter.isJobExisting(jobId)) {
-                    log.info("{} added to executing jobs failed since it already exists", jobId);
-                } else {
-                    execute(jobConfig);
-                }
-                break;
-            default:
-                break;
-        }
-    }
-    
-    private void execute(final JobConfiguration jobConfig) {
-        ConsistencyCheckJob job = new ConsistencyCheckJob();
-        PipelineJobCenter.addJob(jobConfig.getJobName(), job);
-        OneOffJobBootstrap oneOffJobBootstrap = new OneOffJobBootstrap(PipelineAPIFactory.getRegistryCenter(), job, jobConfig);
-        job.setJobBootstrap(oneOffJobBootstrap);
-        oneOffJobBootstrap.execute();
-    }
-    
-    @Override
-    public String getType() {
-        return new ConsistencyCheckJobType().getTypeName();
-    }
-}
diff --git a/kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor b/kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
similarity index 93%
rename from kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor
rename to kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
index bef8b3cefc8..b5b3d90c3d6 100644
--- a/kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor
+++ b/kernel/data-pipeline/scenario/consistencycheck/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.data.pipeline.scenario.consistencycheck.metadata.processor.ConsistencyCheckChangedJobConfigurationProcessor
+org.apache.shardingsphere.data.pipeline.scenario.consistencycheck.metadata.processor.ChangedConsistencyCheckJobConfigurationProcessor
diff --git a/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/ChangedMigrationJobConfigurationProcessor.java b/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/ChangedMigrationJobConfigurationProcessor.java
new file mode 100644
index 00000000000..7937dc8b338
--- /dev/null
+++ b/kernel/data-pipeline/scenario/migration/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/migration/metadata/processor/ChangedMigrationJobConfigurationProcessor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.scenario.migration.metadata.processor;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.data.pipeline.core.job.AbstractPipelineJob;
+import org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.impl.AbstractChangedJobConfigurationProcessor;
+import org.apache.shardingsphere.data.pipeline.scenario.migration.MigrationJob;
+import org.apache.shardingsphere.data.pipeline.scenario.migration.MigrationJobType;
+import org.apache.shardingsphere.data.pipeline.scenario.migration.prepare.MigrationJobPreparer;
+import org.apache.shardingsphere.data.pipeline.spi.job.JobType;
+import org.apache.shardingsphere.data.pipeline.yaml.job.YamlMigrationJobConfigurationSwapper;
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+
+/**
+ * Changed migration job configuration processor.
+ */
+@Slf4j
+public final class ChangedMigrationJobConfigurationProcessor extends AbstractChangedJobConfigurationProcessor {
+    
+    @Override
+    protected void onDeleted(final JobConfiguration jobConfig) {
+        new MigrationJobPreparer().cleanup(new YamlMigrationJobConfigurationSwapper().swapToObject(jobConfig.getJobParameter()));
+    }
+    
+    @Override
+    protected AbstractPipelineJob buildPipelineJob() {
+        return new MigrationJob();
+    }
+    
+    @Override
+    protected JobType getJobType() {
+        return new MigrationJobType();
+    }
+}
diff --git a/kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor b/kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
similarity index 93%
rename from kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor
rename to kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
index 6afc193bf77..c45d5a69c94 100644
--- a/kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.event.handler.PipelineChangedJobConfigurationProcessor
+++ b/kernel/data-pipeline/scenario/migration/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.core.metadata.node.config.processor.ChangedJobConfigurationProcessor
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.data.pipeline.scenario.migration.metadata.processor.MigrationChangedJobConfigurationProcessor
+org.apache.shardingsphere.data.pipeline.scenario.migration.metadata.processor.ChangedMigrationJobConfigurationProcessor