You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/09/29 04:45:30 UTC

[shardingsphere-elasticjob] branch master updated: Distinguish guarantee service by job name in singleton AbstractDistributeOnceElasticJobListener (#1508)

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

zhangliang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere-elasticjob.git


The following commit(s) were added to refs/heads/master by this push:
     new f38acf2  Distinguish guarantee service by job name in singleton AbstractDistributeOnceElasticJobListener (#1508)
f38acf2 is described below

commit f38acf2652db7eafc68b0843efa91b1116e78867
Author: 吴伟杰 <ro...@me.com>
AuthorDate: Tue Sep 29 12:45:22 2020 +0800

    Distinguish guarantee service by job name in singleton AbstractDistributeOnceElasticJobListener (#1508)
    
    * Distinguish guarantee service by job name in singleton AbstractDistributeOnceElasticJobListener
    
    * Fix checkstyle
---
 .../AbstractDistributeOnceElasticJobListener.java  | 68 +++++++++++++++++-----
 .../guarantee/GuaranteeListenerManager.java        |  7 ++-
 .../lite/internal/schedule/JobScheduler.java       | 31 ++++++----
 .../lite/internal/util/ParameterUtils.java         | 50 ++++++++++++++++
 .../DistributeOnceElasticJobListenerTest.java      |  5 +-
 .../TestDistributeOnceElasticJobListener.java      | 13 +----
 .../TestDistributeOnceElasticJobListener.java      |  4 --
 .../guarantee/GuaranteeListenerManagerTest.java    | 12 ++--
 .../util/ParameterUtilsTest.java}                  | 37 ++++++------
 .../fixture/listener/SimpleOnceListener.java       |  1 -
 10 files changed, 160 insertions(+), 68 deletions(-)

diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/listener/AbstractDistributeOnceElasticJobListener.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/listener/AbstractDistributeOnceElasticJobListener.java
index 7ef17f8..ab78fe8 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/listener/AbstractDistributeOnceElasticJobListener.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/listener/AbstractDistributeOnceElasticJobListener.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.elasticjob.lite.api.listener;
 
-import lombok.Setter;
+import lombok.Getter;
 import org.apache.shardingsphere.elasticjob.infra.concurrent.BlockUtils;
 import org.apache.shardingsphere.elasticjob.infra.env.TimeService;
 import org.apache.shardingsphere.elasticjob.infra.exception.JobSystemException;
@@ -26,32 +26,34 @@ import org.apache.shardingsphere.elasticjob.infra.listener.ShardingContexts;
 import org.apache.shardingsphere.elasticjob.lite.internal.guarantee.GuaranteeService;
 
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 /**
  * Distributed once elasticjob listener.
  */
 public abstract class AbstractDistributeOnceElasticJobListener implements ElasticJobListener {
     
-    private final long startedTimeoutMilliseconds;
-    
-    private final Object startedWait = new Object();
-    
-    private final long completedTimeoutMilliseconds;
-    
-    private final Object completedWait = new Object();
-    
-    @Setter
-    private GuaranteeService guaranteeService;
+    private final ConcurrentMap<String, DistributeOnceListenerContext> listenerContexts = new ConcurrentHashMap<>();
     
     private final TimeService timeService = new TimeService();
     
-    public AbstractDistributeOnceElasticJobListener(final long startedTimeoutMilliseconds, final long completedTimeoutMilliseconds) {
-        this.startedTimeoutMilliseconds = startedTimeoutMilliseconds <= 0L ? Long.MAX_VALUE : startedTimeoutMilliseconds;
-        this.completedTimeoutMilliseconds = completedTimeoutMilliseconds <= 0L ? Long.MAX_VALUE : completedTimeoutMilliseconds;
+    /**
+     * Add guarantee service for specific job.
+     *
+     * @param guaranteeService             guarantee service
+     * @param jobName                      job name
+     * @param startedTimeoutMilliseconds   started timeout milliseconds
+     * @param completedTimeoutMilliseconds completed timeout milliseconds
+     */
+    public void addGuaranteeService(final GuaranteeService guaranteeService, final String jobName, final long startedTimeoutMilliseconds, final long completedTimeoutMilliseconds) {
+        listenerContexts.computeIfAbsent(jobName, unused -> new DistributeOnceListenerContext(startedTimeoutMilliseconds, completedTimeoutMilliseconds, guaranteeService));
     }
     
     @Override
     public final void beforeJobExecuted(final ShardingContexts shardingContexts) {
+        DistributeOnceListenerContext context = listenerContexts.get(shardingContexts.getJobName());
+        GuaranteeService guaranteeService = context.getGuaranteeService();
         Set<Integer> shardingItems = shardingContexts.getShardingItemParameters().keySet();
         guaranteeService.registerStart(shardingItems);
         while (!guaranteeService.isRegisterStartSuccess(shardingItems)) {
@@ -63,7 +65,9 @@ public abstract class AbstractDistributeOnceElasticJobListener implements Elasti
             return;
         }
         long before = timeService.getCurrentMillis();
+        long startedTimeoutMilliseconds = context.getStartedTimeoutMilliseconds();
         try {
+            Object startedWait = context.getStartedWait();
             synchronized (startedWait) {
                 startedWait.wait(startedTimeoutMilliseconds);
             }
@@ -78,6 +82,8 @@ public abstract class AbstractDistributeOnceElasticJobListener implements Elasti
     
     @Override
     public final void afterJobExecuted(final ShardingContexts shardingContexts) {
+        DistributeOnceListenerContext context = listenerContexts.get(shardingContexts.getJobName());
+        GuaranteeService guaranteeService = context.getGuaranteeService();
         Set<Integer> shardingItems = shardingContexts.getShardingItemParameters().keySet();
         guaranteeService.registerComplete(shardingItems);
         while (!guaranteeService.isRegisterCompleteSuccess(shardingItems)) {
@@ -89,7 +95,9 @@ public abstract class AbstractDistributeOnceElasticJobListener implements Elasti
             return;
         }
         long before = timeService.getCurrentMillis();
+        long completedTimeoutMilliseconds = context.getCompletedTimeoutMilliseconds();
         try {
+            Object completedWait = context.getCompletedWait();
             synchronized (completedWait) {
                 completedWait.wait(completedTimeoutMilliseconds);
             }
@@ -122,8 +130,12 @@ public abstract class AbstractDistributeOnceElasticJobListener implements Elasti
     
     /**
      * Notify waiting task start.
+     *
+     * @param jobName job name
      */
-    public void notifyWaitingTaskStart() {
+    public void notifyWaitingTaskStart(final String jobName) {
+        DistributeOnceListenerContext context = listenerContexts.get(jobName);
+        Object startedWait = context.getStartedWait();
         synchronized (startedWait) {
             startedWait.notifyAll();
         }
@@ -131,10 +143,34 @@ public abstract class AbstractDistributeOnceElasticJobListener implements Elasti
     
     /**
      * Notify waiting task complete.
+     *
+     * @param jobName job name
      */
-    public void notifyWaitingTaskComplete() {
+    public void notifyWaitingTaskComplete(final String jobName) {
+        DistributeOnceListenerContext context = listenerContexts.get(jobName);
+        Object completedWait = context.getCompletedWait();
         synchronized (completedWait) {
             completedWait.notifyAll();
         }
     }
+    
+    @Getter
+    private static class DistributeOnceListenerContext {
+        
+        private final long startedTimeoutMilliseconds;
+        
+        private final Object startedWait = new Object();
+        
+        private final long completedTimeoutMilliseconds;
+        
+        private final Object completedWait = new Object();
+        
+        private final GuaranteeService guaranteeService;
+        
+        DistributeOnceListenerContext(final long startedTimeoutMilliseconds, final long completedTimeoutMilliseconds, final GuaranteeService guaranteeService) {
+            this.startedTimeoutMilliseconds = startedTimeoutMilliseconds <= 0L ? Long.MAX_VALUE : startedTimeoutMilliseconds;
+            this.completedTimeoutMilliseconds = completedTimeoutMilliseconds <= 0L ? Long.MAX_VALUE : completedTimeoutMilliseconds;
+            this.guaranteeService = guaranteeService;
+        }
+    }
 }
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManager.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManager.java
index 7497592..738dc26 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManager.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManager.java
@@ -34,9 +34,12 @@ public final class GuaranteeListenerManager extends AbstractListenerManager {
     
     private final Collection<ElasticJobListener> elasticJobListeners;
     
+    private final String jobName;
+    
     public GuaranteeListenerManager(final CoordinatorRegistryCenter regCenter, final String jobName, final Collection<ElasticJobListener> elasticJobListeners) {
         super(regCenter, jobName);
         this.guaranteeNode = new GuaranteeNode(jobName);
+        this.jobName = jobName;
         this.elasticJobListeners = elasticJobListeners;
     }
     
@@ -53,7 +56,7 @@ public final class GuaranteeListenerManager extends AbstractListenerManager {
             if (Type.NODE_DELETED == eventType && guaranteeNode.isStartedRootNode(path)) {
                 for (ElasticJobListener each : elasticJobListeners) {
                     if (each instanceof AbstractDistributeOnceElasticJobListener) {
-                        ((AbstractDistributeOnceElasticJobListener) each).notifyWaitingTaskStart();
+                        ((AbstractDistributeOnceElasticJobListener) each).notifyWaitingTaskStart(jobName);
                     }
                 }
             }
@@ -67,7 +70,7 @@ public final class GuaranteeListenerManager extends AbstractListenerManager {
             if (Type.NODE_DELETED == eventType && guaranteeNode.isCompletedRootNode(path)) {
                 for (ElasticJobListener each : elasticJobListeners) {
                     if (each instanceof AbstractDistributeOnceElasticJobListener) {
-                        ((AbstractDistributeOnceElasticJobListener) each).notifyWaitingTaskComplete();
+                        ((AbstractDistributeOnceElasticJobListener) each).notifyWaitingTaskComplete(jobName);
                     }
                 }
             }
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/schedule/JobScheduler.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/schedule/JobScheduler.java
index ab52c6c..c1bf0c3 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/schedule/JobScheduler.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/schedule/JobScheduler.java
@@ -29,6 +29,7 @@ import org.apache.shardingsphere.elasticjob.lite.api.listener.AbstractDistribute
 import org.apache.shardingsphere.elasticjob.lite.internal.guarantee.GuaranteeService;
 import org.apache.shardingsphere.elasticjob.lite.internal.setup.JobClassNameProviderFactory;
 import org.apache.shardingsphere.elasticjob.lite.internal.setup.SetUpFacade;
+import org.apache.shardingsphere.elasticjob.lite.internal.util.ParameterUtils;
 import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
 import org.apache.shardingsphere.elasticjob.tracing.api.TracingConfiguration;
 import org.quartz.JobBuilder;
@@ -39,6 +40,8 @@ import org.quartz.impl.StdSchedulerFactory;
 import org.quartz.simpl.SimpleThreadPool;
 
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
 import java.util.Properties;
 import java.util.stream.Collectors;
 
@@ -76,14 +79,13 @@ public final class JobScheduler {
         this.regCenter = regCenter;
         elasticJobType = null;
         final Collection<ElasticJobListener> elasticJobListeners = jobConfig.getJobListenerTypes().stream()
-                .map(ElasticJobListenerFactory::getListener).collect(Collectors.toList());
+                .map(elasticJobTypeWithParameter -> lookupElasticJobListener(jobConfig.getJobName(), elasticJobTypeWithParameter)).collect(Collectors.toList());
         setUpFacade = new SetUpFacade(regCenter, jobConfig.getJobName(), elasticJobListeners);
         schedulerFacade = new SchedulerFacade(regCenter, jobConfig.getJobName());
         jobFacade = new LiteJobFacade(regCenter, jobConfig.getJobName(), elasticJobListeners, tracingConfig);
         jobExecutor = null == elasticJob ? new ElasticJobExecutor(elasticJobType, jobConfig, jobFacade) : new ElasticJobExecutor(elasticJob, jobConfig, jobFacade);
         String jobClassName = JobClassNameProviderFactory.getProvider().getJobClassName(elasticJob);
         this.jobConfig = setUpFacade.setUpJobConfiguration(jobClassName, jobConfig);
-        setGuaranteeServiceForElasticJobListeners(regCenter, elasticJobListeners);
         jobScheduleController = createJobScheduleController();
     }
     
@@ -95,23 +97,32 @@ public final class JobScheduler {
         this.regCenter = regCenter;
         this.elasticJobType = elasticJobType;
         final Collection<ElasticJobListener> elasticJobListeners = jobConfig.getJobListenerTypes().stream()
-                .map(ElasticJobListenerFactory::getListener).collect(Collectors.toList());
+                .map(elasticJobTypeWithParameter -> lookupElasticJobListener(jobConfig.getJobName(), elasticJobTypeWithParameter)).collect(Collectors.toList());
         setUpFacade = new SetUpFacade(regCenter, jobConfig.getJobName(), elasticJobListeners);
         schedulerFacade = new SchedulerFacade(regCenter, jobConfig.getJobName());
         jobFacade = new LiteJobFacade(regCenter, jobConfig.getJobName(), elasticJobListeners, tracingConfig);
         jobExecutor = new ElasticJobExecutor(elasticJobType, jobConfig, jobFacade);
         this.jobConfig = setUpFacade.setUpJobConfiguration(elasticJobType, jobConfig);
-        setGuaranteeServiceForElasticJobListeners(regCenter, elasticJobListeners);
         jobScheduleController = createJobScheduleController();
     }
     
-    private void setGuaranteeServiceForElasticJobListeners(final CoordinatorRegistryCenter regCenter, final Collection<ElasticJobListener> elasticJobListeners) {
-        GuaranteeService guaranteeService = new GuaranteeService(regCenter, jobConfig.getJobName());
-        for (ElasticJobListener each : elasticJobListeners) {
-            if (each instanceof AbstractDistributeOnceElasticJobListener) {
-                ((AbstractDistributeOnceElasticJobListener) each).setGuaranteeService(guaranteeService);
-            }
+    private ElasticJobListener lookupElasticJobListener(final String jobName, final String jobListenerTypeWithParameter) {
+        String[] split = jobListenerTypeWithParameter.split("\\?");
+        String jobListenerType = split[0];
+        ElasticJobListener listener = ElasticJobListenerFactory.getListener(jobListenerType);
+        if (!(listener instanceof AbstractDistributeOnceElasticJobListener)) {
+            return listener;
         }
+        Map<String, String> parameters = 1 < split.length ? ParameterUtils.parseQuery(split[1]) : Collections.emptyMap();
+        return configureGuaranteeService(jobName, parameters, listener);
+    }
+    
+    private ElasticJobListener configureGuaranteeService(final String jobName, final Map<String, String> parameters, final ElasticJobListener listener) {
+        GuaranteeService guaranteeService = new GuaranteeService(regCenter, jobName);
+        long startedTimeoutMilliseconds = Long.parseLong(parameters.getOrDefault("startedTimeoutMilliseconds", "0"));
+        long completedTimeoutMilliseconds = Long.parseLong(parameters.getOrDefault("completedTimeoutMilliseconds", "0"));
+        ((AbstractDistributeOnceElasticJobListener) listener).addGuaranteeService(guaranteeService, jobName, startedTimeoutMilliseconds, completedTimeoutMilliseconds);
+        return listener;
     }
     
     private JobScheduleController createJobScheduleController() {
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtils.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtils.java
new file mode 100644
index 0000000..cbcb861
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtils.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.elasticjob.lite.internal.util;
+
+import com.google.common.base.Strings;
+import lombok.AccessLevel;
+import lombok.RequiredArgsConstructor;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Parameter utils.
+ */
+@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ParameterUtils {
+    
+    /**
+     * Parse string like <code>key1=value1&amp;key2=value2</code> to {@link Map}.
+     *
+     * @param query parameter string
+     * @return map
+     */
+    public static Map<String, String> parseQuery(final String query) {
+        if (Strings.isNullOrEmpty(query)) {
+            return Collections.emptyMap();
+        }
+        return Arrays.stream(query.split("&")).map(String::trim)
+                .filter(pair -> !pair.isEmpty())
+                .map(parameter -> parameter.split("="))
+                .collect(Collectors.toMap(pair -> pair[0], pair -> 1 < pair.length ? pair[1] : ""));
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/DistributeOnceElasticJobListenerTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/DistributeOnceElasticJobListenerTest.java
index 549daef..bfeef4c 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/DistributeOnceElasticJobListenerTest.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/DistributeOnceElasticJobListenerTest.java
@@ -57,8 +57,9 @@ public final class DistributeOnceElasticJobListenerTest {
     
     @Before
     public void setUp() {
-        distributeOnceElasticJobListener = new TestDistributeOnceElasticJobListener(elasticJobListenerCaller);
-        distributeOnceElasticJobListener.setGuaranteeService(guaranteeService);
+        distributeOnceElasticJobListener = new TestDistributeOnceElasticJobListener();
+        distributeOnceElasticJobListener.setCaller(elasticJobListenerCaller);
+        distributeOnceElasticJobListener.addGuaranteeService(guaranteeService, "test_job", 1L, 1L);
         ReflectionUtils.setSuperclassFieldValue(distributeOnceElasticJobListener, "timeService", timeService);
         Map<Integer, String> map = new HashMap<>(2, 1);
         map.put(0, "");
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/fixture/TestDistributeOnceElasticJobListener.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/fixture/TestDistributeOnceElasticJobListener.java
index 6150d65..be05179 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/fixture/TestDistributeOnceElasticJobListener.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/api/listener/fixture/TestDistributeOnceElasticJobListener.java
@@ -17,21 +17,14 @@
 
 package org.apache.shardingsphere.elasticjob.lite.api.listener.fixture;
 
+import lombok.Setter;
 import org.apache.shardingsphere.elasticjob.infra.listener.ShardingContexts;
 import org.apache.shardingsphere.elasticjob.lite.api.listener.AbstractDistributeOnceElasticJobListener;
 
 public final class TestDistributeOnceElasticJobListener extends AbstractDistributeOnceElasticJobListener {
     
-    private final ElasticJobListenerCaller caller;
-    
-    public TestDistributeOnceElasticJobListener() {
-        this(null);
-    }
-    
-    public TestDistributeOnceElasticJobListener(final ElasticJobListenerCaller caller) {
-        super(1L, 1L);
-        this.caller = caller;
-    }
+    @Setter
+    private ElasticJobListenerCaller caller;
     
     @Override
     public void doBeforeJobExecutedAtLastStarted(final ShardingContexts shardingContexts) {
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java
index 5bcb7f8..c2c1ef7 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java
@@ -22,10 +22,6 @@ import org.apache.shardingsphere.elasticjob.lite.api.listener.AbstractDistribute
 
 public class TestDistributeOnceElasticJobListener extends AbstractDistributeOnceElasticJobListener {
     
-    public TestDistributeOnceElasticJobListener() {
-        super(100L, 100L);
-    }
-    
     @Override
     public void doBeforeJobExecutedAtLastStarted(final ShardingContexts shardingContexts) {
     }
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManagerTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManagerTest.java
index ca2be42..279379a 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManagerTest.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/guarantee/GuaranteeListenerManagerTest.java
@@ -64,36 +64,36 @@ public final class GuaranteeListenerManagerTest {
     @Test
     public void assertStartedNodeRemovedJobListenerWhenIsNotRemoved() {
         guaranteeListenerManager.new StartedNodeRemovedJobListener().dataChanged("/test_job/guarantee/started", Type.NODE_CHANGED, "");
-        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart();
+        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart("test_job");
     }
     
     @Test
     public void assertStartedNodeRemovedJobListenerWhenIsNotStartedNode() {
         guaranteeListenerManager.new StartedNodeRemovedJobListener().dataChanged("/other_job/guarantee/started", Type.NODE_DELETED, "");
-        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart();
+        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart("test_job");
     }
     
     @Test
     public void assertStartedNodeRemovedJobListenerWhenIsRemovedAndStartedNode() {
         guaranteeListenerManager.new StartedNodeRemovedJobListener().dataChanged("/test_job/guarantee/started", Type.NODE_DELETED, "");
-        verify(distributeOnceElasticJobListener).notifyWaitingTaskStart();
+        verify(distributeOnceElasticJobListener).notifyWaitingTaskStart("test_job");
     }
     
     @Test
     public void assertCompletedNodeRemovedJobListenerWhenIsNotRemoved() {
         guaranteeListenerManager.new CompletedNodeRemovedJobListener().dataChanged("/test_job/guarantee/completed", Type.NODE_CHANGED, "");
-        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart();
+        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart("test_job");
     }
     
     @Test
     public void assertCompletedNodeRemovedJobListenerWhenIsNotCompletedNode() {
         guaranteeListenerManager.new CompletedNodeRemovedJobListener().dataChanged("/other_job/guarantee/completed", Type.NODE_DELETED, "");
-        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart();
+        verify(distributeOnceElasticJobListener, times(0)).notifyWaitingTaskStart("test_job");
     }
     
     @Test
     public void assertCompletedNodeRemovedJobListenerWhenIsRemovedAndCompletedNode() {
         guaranteeListenerManager.new CompletedNodeRemovedJobListener().dataChanged("/test_job/guarantee/completed", Type.NODE_DELETED, "");
-        verify(distributeOnceElasticJobListener).notifyWaitingTaskComplete();
+        verify(distributeOnceElasticJobListener).notifyWaitingTaskComplete("test_job");
     }
 }
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtilsTest.java
similarity index 51%
copy from elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java
copy to elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtilsTest.java
index 5bcb7f8..63d737e 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/integrate/listener/TestDistributeOnceElasticJobListener.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/util/ParameterUtilsTest.java
@@ -15,27 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.elasticjob.lite.integrate.listener;
+package org.apache.shardingsphere.elasticjob.lite.internal.util;
 
-import org.apache.shardingsphere.elasticjob.infra.listener.ShardingContexts;
-import org.apache.shardingsphere.elasticjob.lite.api.listener.AbstractDistributeOnceElasticJobListener;
+import org.junit.Test;
 
-public class TestDistributeOnceElasticJobListener extends AbstractDistributeOnceElasticJobListener {
-    
-    public TestDistributeOnceElasticJobListener() {
-        super(100L, 100L);
-    }
-    
-    @Override
-    public void doBeforeJobExecutedAtLastStarted(final ShardingContexts shardingContexts) {
-    }
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ParameterUtilsTest {
     
-    @Override
-    public void doAfterJobExecutedAtLastCompleted(final ShardingContexts shardingContexts) {
+    @Test
+    public void assertParseQueryString() {
+        String queryString = "key1=foo&key2&key3=bar";
+        Map<String, String> result = ParameterUtils.parseQuery(queryString);
+        assertThat(result.get("key1"), is("foo"));
+        assertThat(result.get("key2"), is(""));
+        assertThat(result.get("key3"), is("bar"));
     }
     
-    @Override
-    public String getType() {
-        return "INTEGRATE-DISTRIBUTE";
+    @Test
+    public void assertParseEmptyString() {
+        Map<String, String> result = ParameterUtils.parseQuery("");
+        assertTrue(result.isEmpty());
     }
 }
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/listener/SimpleOnceListener.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/listener/SimpleOnceListener.java
index b7dc213..67e7859 100644
--- a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/listener/SimpleOnceListener.java
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/listener/SimpleOnceListener.java
@@ -40,7 +40,6 @@ public class SimpleOnceListener extends AbstractDistributeOnceElasticJobListener
     }
     
     public SimpleOnceListener(final long startedTimeoutMilliseconds, final long completedTimeoutMilliseconds) {
-        super(startedTimeoutMilliseconds, completedTimeoutMilliseconds);
         this.startedTimeoutMilliseconds = startedTimeoutMilliseconds;
         this.completedTimeoutMilliseconds = completedTimeoutMilliseconds;
     }