You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by wu...@apache.org on 2022/12/21 08:55:40 UTC

[shardingsphere-elasticjob] branch master updated: Support annotation job (#2103)

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

wuweijie 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 666b7c977 Support annotation job  (#2103)
666b7c977 is described below

commit 666b7c977f7a66789fd0fe15dcaf86fc0385a47d
Author: skai <su...@gmail.com>
AuthorDate: Wed Dec 21 16:55:30 2022 +0800

    Support annotation job  (#2103)
    
    * add annotation interface
    
    * add test
    
    * add JobAnnotationBuilder
    
    * complete JobAnntotationBuilder
    
    * Bootstarp support annotation job && add test
    
    * add License
    
    * fix bug
    
    * fix review problem
    
    * import `Optional`
    
    * fix bug
    
    * spring scan the elasticJobConfiguration
    
    * move to spring-core
    
    * add test
    
    * fix from review
    
    * fix again
    
    * Update elasticjob.xsd
    
     revert these format changes
    
    * Update JobScannerConfiguration.java
    
    simplified by @RequiredArgsConstructor
    
    * remove implements ResourceLoaderAware
    
    Co-authored-by: 蔡顺铠 <sk...@gmail.com>
    Co-authored-by: skcai <sk...@topode.com>
---
 .../annotation/ElasticJobConfiguration.java        | 161 +++++++++++++++++++++
 .../elasticjob/annotation/ElasticJobProp.java      |  42 ++++++
 .../api/JobExtraConfigurationFactory.java          |  32 ++++
 .../annotation/ElasticJobConfigurationTest.java    |  51 +++++++
 .../SimpleTracingConfigurationFactory.java         |  30 ++++
 .../elasticjob/annotation/job/CustomJob.java       |  31 ++++
 .../annotation/job/impl/SimpleTestJob.java         |  43 ++++++
 .../api/bootstrap/impl/OneOffJobBootstrap.java     |   7 +
 .../api/bootstrap/impl/ScheduleJobBootstrap.java   |   6 +
 .../internal/annotation/JobAnnotationBuilder.java  |  74 ++++++++++
 .../lite/fixture/job/AnnotationSimpleJob.java      |  46 ++++++
 .../lite/fixture/job/AnnotationUnShardingJob.java  |  39 +++++
 .../annotation/JobAnnotationBuilderTest.java       |  54 +++++++
 .../annotation/integrate/BaseAnnotationTest.java   | 102 +++++++++++++
 .../annotation/integrate/OneOffEnabledJobTest.java |  64 ++++++++
 .../integrate/ScheduleEnabledJobTest.java          |  66 +++++++++
 .../boot/job/ElasticJobSpringBootScannerTest.java  |  54 +++++++
 .../job/fixture/job/impl/AnnotationCustomJob.java  |  58 ++++++++
 .../spring/core/scanner/ClassPathJobScanner.java   |  85 +++++++++++
 .../lite/spring/core/scanner/ElasticJobScan.java   |  51 +++++++
 .../core/scanner/ElasticJobScanRegistrar.java      |  65 +++++++++
 .../core/scanner/JobScannerConfiguration.java      |  55 +++++++
 .../namespace/ElasticJobNamespaceHandler.java      |   2 +
 .../parser/JobScannerBeanDefinitionParser.java     |  40 +++++
 .../scanner/tag/JobScannerBeanDefinitionTag.java   |  29 ++++
 .../resources/META-INF/namespace/elasticjob.xsd    |  21 ++-
 .../job/annotation/AnnotationSimpleJob.java        |  55 +++++++
 .../scanner/AbstractJobSpringIntegrateTest.java    |  65 +++++++++
 .../spring/namespace/scanner/JobScannerTest.java   |  28 ++++
 .../META-INF/scanner/jobScannerContext.xml         |  36 +++++
 30 files changed, 1487 insertions(+), 5 deletions(-)

diff --git a/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfiguration.java b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfiguration.java
new file mode 100644
index 000000000..7b7e841f8
--- /dev/null
+++ b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfiguration.java
@@ -0,0 +1,161 @@
+/*
+ * 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.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfigurationFactory;
+
+/**
+ * The annotation that specify a job of elastic.
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface ElasticJobConfiguration {
+    
+    /**
+     * Job name.
+     * @return jobName
+     */
+    String jobName();
+    
+    /**
+     * CRON expression, control the job trigger time.
+     * @return cron
+     */
+    String cron() default "";
+    
+    /**
+     * Time zone of CRON.
+     * @return timeZone
+     */
+    String timeZone() default "";
+
+
+    /**
+     * registry center name.
+     * @return registryCenter
+     */
+    String registryCenter() default "";
+
+    /**
+     * Sharding total count.
+     * @return shardingTotalCount
+     */
+    int shardingTotalCount();
+    
+    /**
+     * Sharding item parameters.
+     * @return shardingItemParameters
+     */
+    String shardingItemParameters() default "";
+    
+    /**
+     * Job parameter.
+     * @return jobParameter
+     */
+    String jobParameter() default "";
+    
+    /**
+     * Monitor job execution status.
+     * @return monitorExecution
+     */
+    boolean monitorExecution() default true;
+    
+    /**
+     * Enable or disable job failover.
+     * @return failover
+     */
+    boolean failover() default false;
+    
+    /**
+     * Enable or disable the missed task to re-execute.
+     * @return misfire
+     */
+    boolean misfire() default true;
+    
+    /**
+     * The maximum value for time difference between server and registry center in seconds.
+     * @return maxTimeDiffSeconds
+     */
+    int maxTimeDiffSeconds() default -1;
+    
+    /**
+     * Service scheduling interval in minutes for repairing job server inconsistent state.
+     * @return reconcileIntervalMinutes
+     */
+    int reconcileIntervalMinutes() default 10;
+    
+    /**
+     * Job sharding strategy type.
+     * @return jobShardingStrategyType
+     */
+    String jobShardingStrategyType() default "";
+    
+    /**
+     * Job thread pool handler type.
+     * @return jobExecutorServiceHandlerType
+     */
+    String jobExecutorServiceHandlerType() default "";
+    
+    /**
+     * Job thread pool handler type.
+     * @return jobErrorHandlerType
+     */
+    String jobErrorHandlerType() default "";
+    
+    /**
+     * Job listener types.
+     * @return jobListenerTypes
+     */
+    String[] jobListenerTypes() default {};
+    
+    /**
+     * extra configurations.
+     * @return extraConfigurations
+     */
+    Class<? extends JobExtraConfigurationFactory>[] extraConfigurations() default {};
+    
+    /**
+     * Job description.
+     * @return description
+     */
+    String description() default "";
+    
+    /**
+     * Job properties.
+     * @return props
+     */
+    ElasticJobProp[] props() default {};
+    
+    /**
+     * Enable or disable start the job.
+     * @return disabled
+     */
+    boolean disabled() default false;
+    
+    /**
+     * Enable or disable local configuration override registry center configuration.
+     * @return overwrite
+     */
+    boolean overwrite() default false;
+}
diff --git a/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobProp.java b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobProp.java
new file mode 100644
index 000000000..aa0d7904e
--- /dev/null
+++ b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobProp.java
@@ -0,0 +1,42 @@
+/*
+ * 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.annotation;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * The annotation that specify elastic-job prop.
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+public @interface ElasticJobProp {
+    
+    /**
+     * Prop key.
+     * @return key
+     */
+    String key();
+    
+    /**
+     * Prop value.
+     * @return value
+     */
+    String value() default "";
+}
diff --git a/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/api/JobExtraConfigurationFactory.java b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/api/JobExtraConfigurationFactory.java
new file mode 100644
index 000000000..8670ed93a
--- /dev/null
+++ b/elasticjob-api/src/main/java/org/apache/shardingsphere/elasticjob/api/JobExtraConfigurationFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.api;
+
+import java.util.Optional;
+
+/**
+ * Job extra configuration factory.
+ */
+public interface JobExtraConfigurationFactory {
+    
+    /**
+     * Get JobExtraConfiguration.
+     * @return JobExtraConfiguration
+     */
+    Optional<JobExtraConfiguration> getJobExtraConfiguration();
+}
diff --git a/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfigurationTest.java b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfigurationTest.java
new file mode 100644
index 000000000..ecba0802e
--- /dev/null
+++ b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/ElasticJobConfigurationTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.annotation;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertThat;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.Queue;
+import org.apache.shardingsphere.elasticjob.annotation.job.impl.SimpleTestJob;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfigurationFactory;
+import org.junit.Test;
+
+public final class ElasticJobConfigurationTest {
+    
+    @Test
+    public void assertAnnotationJob() {
+        ElasticJobConfiguration annotation = SimpleTestJob.class.getAnnotation(ElasticJobConfiguration.class);
+        assertThat(annotation.jobName(), is("SimpleTestJob"));
+        assertThat(annotation.cron(), is("0/5 * * * * ?"));
+        assertThat(annotation.shardingTotalCount(), is(3));
+        assertThat(annotation.shardingItemParameters(), is("0=Beijing,1=Shanghai,2=Guangzhou"));
+        for (Class<? extends JobExtraConfigurationFactory> factory :annotation.extraConfigurations()) {
+            assertThat(factory, is(SimpleTracingConfigurationFactory.class));
+        }
+        assertArrayEquals(annotation.jobListenerTypes(), new String[] {"NOOP", "LOG"});
+        Queue<String> propsKey = new LinkedList<>(Arrays.asList("print.title", "print.content"));
+        Queue<String> propsValue = new LinkedList<>(Arrays.asList("test title", "test content"));
+        for (ElasticJobProp prop :annotation.props()) {
+            assertThat(prop.key(), is(propsKey.poll()));
+            assertThat(prop.value(), is(propsValue.poll()));
+        }
+    }
+}
diff --git a/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/SimpleTracingConfigurationFactory.java b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/SimpleTracingConfigurationFactory.java
new file mode 100644
index 000000000..cbc0b8de2
--- /dev/null
+++ b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/SimpleTracingConfigurationFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.annotation;
+
+import java.util.Optional;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfiguration;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfigurationFactory;
+
+public final class SimpleTracingConfigurationFactory implements JobExtraConfigurationFactory {
+    
+    @Override
+    public Optional<JobExtraConfiguration> getJobExtraConfiguration() {
+        return Optional.empty();
+    }
+}
diff --git a/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/CustomJob.java b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/CustomJob.java
new file mode 100644
index 000000000..fbad39c62
--- /dev/null
+++ b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/CustomJob.java
@@ -0,0 +1,31 @@
+/*
+ * 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.annotation.job;
+
+import org.apache.shardingsphere.elasticjob.api.ElasticJob;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+
+public interface CustomJob extends ElasticJob {
+    
+    /**
+     * Execute custom job.
+     *
+     * @param shardingContext sharding context
+     */
+    void execute(ShardingContext shardingContext);
+}
diff --git a/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/impl/SimpleTestJob.java b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/impl/SimpleTestJob.java
new file mode 100644
index 000000000..1c85c7832
--- /dev/null
+++ b/elasticjob-api/src/test/java/org/apache/shardingsphere/elasticjob/annotation/job/impl/SimpleTestJob.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.elasticjob.annotation.job.impl;
+
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobProp;
+import org.apache.shardingsphere.elasticjob.annotation.SimpleTracingConfigurationFactory;
+import org.apache.shardingsphere.elasticjob.annotation.job.CustomJob;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+
+@ElasticJobConfiguration(
+    cron = "0/5 * * * * ?",
+    jobName = "SimpleTestJob",
+    shardingTotalCount = 3,
+    shardingItemParameters = "0=Beijing,1=Shanghai,2=Guangzhou",
+    jobListenerTypes = {"NOOP", "LOG"},
+    extraConfigurations = {SimpleTracingConfigurationFactory.class},
+    props = {
+        @ElasticJobProp(key = "print.title", value = "test title"),
+        @ElasticJobProp(key = "print.content", value = "test content")
+    }
+)
+public final class SimpleTestJob implements CustomJob {
+    
+    @Override
+    public void execute(final ShardingContext shardingContext) {
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/OneOffJobBootstrap.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/OneOffJobBootstrap.java
index a98a33e3f..3423f2470 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/OneOffJobBootstrap.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/OneOffJobBootstrap.java
@@ -22,6 +22,7 @@ import com.google.common.base.Strings;
 import org.apache.shardingsphere.elasticjob.api.ElasticJob;
 import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
 import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.JobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.internal.annotation.JobAnnotationBuilder;
 import org.apache.shardingsphere.elasticjob.lite.internal.instance.InstanceService;
 import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobScheduler;
 import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
@@ -47,6 +48,12 @@ public final class OneOffJobBootstrap implements JobBootstrap {
         instanceService = new InstanceService(regCenter, jobConfig.getJobName());
     }
     
+    public OneOffJobBootstrap(final CoordinatorRegistryCenter regCenter, final ElasticJob elasticJob) {
+        JobConfiguration jobConfig = JobAnnotationBuilder.generateJobConfiguration(elasticJob.getClass());
+        jobScheduler = new JobScheduler(regCenter, elasticJob, jobConfig);
+        instanceService = new InstanceService(regCenter, jobConfig.getJobName());
+    }
+    
     /**
      * Execute job.
      */
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/ScheduleJobBootstrap.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/ScheduleJobBootstrap.java
index bb491995e..8d1f6cee9 100644
--- a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/ScheduleJobBootstrap.java
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/api/bootstrap/impl/ScheduleJobBootstrap.java
@@ -22,6 +22,7 @@ import com.google.common.base.Strings;
 import org.apache.shardingsphere.elasticjob.api.ElasticJob;
 import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
 import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.JobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.internal.annotation.JobAnnotationBuilder;
 import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobScheduler;
 import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
 
@@ -40,6 +41,11 @@ public final class ScheduleJobBootstrap implements JobBootstrap {
         jobScheduler = new JobScheduler(regCenter, elasticJobType, jobConfig);
     }
     
+    public ScheduleJobBootstrap(final CoordinatorRegistryCenter regCenter, final ElasticJob elasticJob) {
+        JobConfiguration jobConfig = JobAnnotationBuilder.generateJobConfiguration(elasticJob.getClass());
+        jobScheduler = new JobScheduler(regCenter, elasticJob, jobConfig);
+    }
+    
     /**
      * Schedule job.
      */
diff --git a/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilder.java b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilder.java
new file mode 100644
index 000000000..6353b9683
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilder.java
@@ -0,0 +1,74 @@
+/*
+ * 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.annotation;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import java.util.Optional;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobProp;
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfiguration;
+import org.apache.shardingsphere.elasticjob.api.JobExtraConfigurationFactory;
+import org.apache.shardingsphere.elasticjob.infra.exception.JobConfigurationException;
+
+/**
+ * Job Builder from @ElasticJobConfiguration.
+ */
+public final class JobAnnotationBuilder {
+    
+    /**
+     * generate JobConfiguration from @ElasticJobConfiguration.
+     * @param type The job of @ElasticJobConfiguration annotation class
+     * @return JobConfiguration
+     */
+    public static JobConfiguration generateJobConfiguration(final Class<?> type) {
+        ElasticJobConfiguration annotation = type.getAnnotation(ElasticJobConfiguration.class);
+        Preconditions.checkArgument(null != annotation, "@ElasticJobConfiguration not found by class '%s'.", type);
+        Preconditions.checkArgument(!Strings.isNullOrEmpty(annotation.jobName()), "@ElasticJobConfiguration jobName could not be empty by class '%s'.", type);
+        JobConfiguration.Builder jobConfigurationBuilder = JobConfiguration.newBuilder(annotation.jobName(), annotation.shardingTotalCount())
+                .shardingItemParameters(annotation.shardingItemParameters())
+                .cron(Strings.isNullOrEmpty(annotation.cron()) ? null : annotation.cron())
+                .timeZone(Strings.isNullOrEmpty(annotation.timeZone()) ? null : annotation.timeZone())
+                .jobParameter(annotation.jobParameter())
+                .monitorExecution(annotation.monitorExecution())
+                .failover(annotation.failover())
+                .misfire(annotation.misfire())
+                .maxTimeDiffSeconds(annotation.maxTimeDiffSeconds())
+                .reconcileIntervalMinutes(annotation.reconcileIntervalMinutes())
+                .jobShardingStrategyType(Strings.isNullOrEmpty(annotation.jobShardingStrategyType()) ? null : annotation.jobShardingStrategyType())
+                .jobExecutorServiceHandlerType(Strings.isNullOrEmpty(annotation.jobExecutorServiceHandlerType()) ? null : annotation.jobExecutorServiceHandlerType())
+                .jobErrorHandlerType(Strings.isNullOrEmpty(annotation.jobErrorHandlerType()) ? null : annotation.jobErrorHandlerType())
+                .jobListenerTypes(annotation.jobListenerTypes())
+                .description(annotation.description())
+                .disabled(annotation.disabled())
+                .overwrite(annotation.overwrite());
+        for (Class<? extends JobExtraConfigurationFactory> clazz : annotation.extraConfigurations()) {
+            try {
+                Optional<JobExtraConfiguration> jobExtraConfiguration = clazz.newInstance().getJobExtraConfiguration();
+                jobExtraConfiguration.ifPresent(jobConfigurationBuilder::addExtraConfigurations);
+            } catch (IllegalAccessException | InstantiationException exception) {
+                throw (JobConfigurationException) new JobConfigurationException("new JobExtraConfigurationFactory instance by class '%s' failure", clazz).initCause(exception);
+            }
+        }
+        for (ElasticJobProp prop :annotation.props()) {
+            jobConfigurationBuilder.setProperty(prop.key(), prop.value());
+        }
+        return jobConfigurationBuilder.build();
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationSimpleJob.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationSimpleJob.java
new file mode 100644
index 000000000..6d3391568
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationSimpleJob.java
@@ -0,0 +1,46 @@
+/*
+ * 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.fixture.job;
+
+import lombok.Getter;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobProp;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+import org.apache.shardingsphere.elasticjob.simple.job.SimpleJob;
+
+@Getter
+@ElasticJobConfiguration(
+    jobName = "AnnotationSimpleJob",
+    description = "desc",
+    shardingTotalCount = 3,
+    shardingItemParameters = "0=a,1=b,2=c",
+    cron = "*/10 * * * * ?",
+    props = {
+        @ElasticJobProp(key = "print.title", value = "test title"),
+        @ElasticJobProp(key = "print.content", value = "test content")
+    }
+)
+public class AnnotationSimpleJob implements SimpleJob {
+    
+    private volatile boolean completed;
+    
+    @Override
+    public void execute(final ShardingContext shardingContext) {
+        completed = true;
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationUnShardingJob.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationUnShardingJob.java
new file mode 100644
index 000000000..aab19658b
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/fixture/job/AnnotationUnShardingJob.java
@@ -0,0 +1,39 @@
+/*
+ * 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.fixture.job;
+
+import lombok.Getter;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+import org.apache.shardingsphere.elasticjob.simple.job.SimpleJob;
+
+@Getter
+@ElasticJobConfiguration(
+    jobName = "AnnotationUnShardingJob",
+    description = "desc",
+    shardingTotalCount = 1
+)
+public final class AnnotationUnShardingJob implements SimpleJob {
+    
+    private volatile boolean completed;
+    
+    @Override
+    public void execute(final ShardingContext shardingContext) {
+        completed = true;
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilderTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilderTest.java
new file mode 100644
index 000000000..25b73e555
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/JobAnnotationBuilderTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.annotation;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+import org.apache.shardingsphere.elasticjob.lite.fixture.job.AnnotationSimpleJob;
+import org.junit.Test;
+
+public final class JobAnnotationBuilderTest {
+   
+    @Test
+    public void assertGenerateJobConfiguration() {
+        JobConfiguration jobConfiguration = JobAnnotationBuilder.generateJobConfiguration(AnnotationSimpleJob.class);
+        assertThat(jobConfiguration.getJobName(), is("AnnotationSimpleJob"));
+        assertThat(jobConfiguration.getShardingTotalCount(), is(3));
+        assertThat(jobConfiguration.getShardingItemParameters(), is("0=a,1=b,2=c"));
+        assertThat(jobConfiguration.getCron(), is("*/10 * * * * ?"));
+        assertTrue(jobConfiguration.isMonitorExecution());
+        assertFalse(jobConfiguration.isFailover());
+        assertTrue(jobConfiguration.isMisfire());
+        assertThat(jobConfiguration.getMaxTimeDiffSeconds(), is(-1));
+        assertThat(jobConfiguration.getReconcileIntervalMinutes(), is(10));
+        assertNull(jobConfiguration.getJobShardingStrategyType());
+        assertNull(jobConfiguration.getJobExecutorServiceHandlerType());
+        assertNull(jobConfiguration.getJobErrorHandlerType());
+        assertThat(jobConfiguration.getDescription(), is("desc"));
+        assertThat(jobConfiguration.getProps().getProperty("print.title"), is("test title"));
+        assertThat(jobConfiguration.getProps().getProperty("print.content"), is("test content"));
+        assertFalse(jobConfiguration.isDisabled());
+        assertFalse(jobConfiguration.isOverwrite());
+    }
+
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/BaseAnnotationTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/BaseAnnotationTest.java
new file mode 100644
index 000000000..bbc3e46cd
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/BaseAnnotationTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.annotation.integrate;
+
+import lombok.AccessLevel;
+import lombok.Getter;
+import org.apache.shardingsphere.elasticjob.api.ElasticJob;
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.JobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.OneOffJobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.ScheduleJobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.fixture.EmbedTestingServer;
+import org.apache.shardingsphere.elasticjob.lite.internal.annotation.JobAnnotationBuilder;
+import org.apache.shardingsphere.elasticjob.lite.internal.election.LeaderService;
+import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobRegistry;
+import org.apache.shardingsphere.elasticjob.lite.util.ReflectionUtils;
+import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
+import org.apache.shardingsphere.elasticjob.reg.zookeeper.ZookeeperConfiguration;
+import org.apache.shardingsphere.elasticjob.reg.zookeeper.ZookeeperRegistryCenter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+@Getter(AccessLevel.PROTECTED)
+public abstract class BaseAnnotationTest {
+    
+    private static final ZookeeperConfiguration ZOOKEEPER_CONFIG = new ZookeeperConfiguration(EmbedTestingServer.getConnectionString(), "zkRegTestCenter");
+    
+    @Getter(AccessLevel.PROTECTED)
+    private static final CoordinatorRegistryCenter REGISTRY_CENTER = new ZookeeperRegistryCenter(ZOOKEEPER_CONFIG);
+    
+    private final ElasticJob elasticJob;
+            
+    private final JobConfiguration jobConfiguration;
+    
+    private final JobBootstrap jobBootstrap;
+    
+    private final LeaderService leaderService;
+    
+    private final String jobName;
+    
+    protected BaseAnnotationTest(final TestType type, final ElasticJob elasticJob) {
+        this.elasticJob = elasticJob;
+        jobConfiguration = JobAnnotationBuilder.generateJobConfiguration(elasticJob.getClass());
+        jobName = jobConfiguration.getJobName();
+        jobBootstrap = createJobBootstrap(type, elasticJob);
+        leaderService = new LeaderService(REGISTRY_CENTER, jobName);
+    }
+    
+    private JobBootstrap createJobBootstrap(final TestType type, final ElasticJob elasticJob) {
+        switch (type) {
+            case SCHEDULE:
+                return new ScheduleJobBootstrap(REGISTRY_CENTER, elasticJob);
+            case ONE_OFF:
+                return new OneOffJobBootstrap(REGISTRY_CENTER, elasticJob);
+            default:
+                throw new RuntimeException(String.format("Cannot support `%s`", type));
+        }
+    }
+    
+    @BeforeClass
+    public static void init() {
+        EmbedTestingServer.start();
+        ZOOKEEPER_CONFIG.setConnectionTimeoutMilliseconds(30000);
+        REGISTRY_CENTER.init();
+    }
+    
+    @Before
+    public void setUp() {
+        if (jobBootstrap instanceof ScheduleJobBootstrap) {
+            ((ScheduleJobBootstrap) jobBootstrap).schedule();
+        } else {
+            ((OneOffJobBootstrap) jobBootstrap).execute();
+        }
+    }
+    
+    @After
+    public void tearDown() {
+        jobBootstrap.shutdown();
+        ReflectionUtils.setFieldValue(JobRegistry.getInstance(), "instance", null);
+    }
+    
+    public enum TestType {
+        
+        SCHEDULE, ONE_OFF
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/OneOffEnabledJobTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/OneOffEnabledJobTest.java
new file mode 100644
index 000000000..33bf40ef8
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/OneOffEnabledJobTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.annotation.integrate;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+import org.apache.shardingsphere.elasticjob.infra.concurrent.BlockUtils;
+import org.apache.shardingsphere.elasticjob.infra.env.IpUtils;
+import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO;
+import org.apache.shardingsphere.elasticjob.infra.yaml.YamlEngine;
+import org.apache.shardingsphere.elasticjob.lite.fixture.job.AnnotationUnShardingJob;
+import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobRegistry;
+import org.apache.shardingsphere.elasticjob.lite.internal.server.ServerStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+public final class OneOffEnabledJobTest extends BaseAnnotationTest {
+    
+    public OneOffEnabledJobTest() {
+        super(TestType.ONE_OFF, new AnnotationUnShardingJob());
+    }
+    
+    @Before
+    public void assertEnabledRegCenterInfo() {
+        assertThat(JobRegistry.getInstance().getCurrentShardingTotalCount(getJobName()), is(1));
+        assertThat(JobRegistry.getInstance().getJobInstance(getJobName()).getServerIp(), is(IpUtils.getIp()));
+        JobConfiguration jobConfig = YamlEngine.unmarshal(getREGISTRY_CENTER().get("/" + getJobName() + "/config"), JobConfigurationPOJO.class).toJobConfiguration();
+        assertThat(jobConfig.getShardingTotalCount(), is(1));
+        assertNull(jobConfig.getCron());
+        assertThat(getREGISTRY_CENTER().get("/" + getJobName() + "/servers/" + JobRegistry.getInstance().getJobInstance(getJobName()).getServerIp()), is(ServerStatus.ENABLED.name()));
+        assertThat(getREGISTRY_CENTER().get("/" + getJobName() + "/leader/election/instance"), is(JobRegistry.getInstance().getJobInstance(getJobName()).getJobInstanceId()));
+        assertTrue(getREGISTRY_CENTER().isExisted("/" + getJobName() + "/instances/" + JobRegistry.getInstance().getJobInstance(getJobName()).getJobInstanceId()));
+        getREGISTRY_CENTER().remove("/" + getJobName() + "/leader/election");
+        assertTrue(getLeaderService().isLeaderUntilBlock());
+    }
+    
+    @Test
+    public void assertJobInit() {
+        while (!((AnnotationUnShardingJob) getElasticJob()).isCompleted()) {
+            BlockUtils.waitingShortTime();
+        }
+        assertTrue(getREGISTRY_CENTER().isExisted("/" + getJobName() + "/sharding"));
+    }
+    
+}
diff --git a/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/ScheduleEnabledJobTest.java b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/ScheduleEnabledJobTest.java
new file mode 100644
index 000000000..d250341e3
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-core/src/test/java/org/apache/shardingsphere/elasticjob/lite/internal/annotation/integrate/ScheduleEnabledJobTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.annotation.integrate;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.shardingsphere.elasticjob.api.JobConfiguration;
+import org.apache.shardingsphere.elasticjob.infra.concurrent.BlockUtils;
+import org.apache.shardingsphere.elasticjob.infra.env.IpUtils;
+import org.apache.shardingsphere.elasticjob.infra.pojo.JobConfigurationPOJO;
+import org.apache.shardingsphere.elasticjob.infra.yaml.YamlEngine;
+import org.apache.shardingsphere.elasticjob.lite.fixture.job.AnnotationSimpleJob;
+import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobRegistry;
+import org.apache.shardingsphere.elasticjob.lite.internal.server.ServerStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+public final class ScheduleEnabledJobTest extends BaseAnnotationTest {
+    
+    public ScheduleEnabledJobTest() {
+        super(TestType.SCHEDULE, new AnnotationSimpleJob());
+    }
+    
+    @Before
+    public void assertEnabledRegCenterInfo() {
+        assertThat(JobRegistry.getInstance().getCurrentShardingTotalCount(getJobName()), is(3));
+        assertThat(JobRegistry.getInstance().getJobInstance(getJobName()).getServerIp(), is(IpUtils.getIp()));
+        JobConfiguration jobConfig = YamlEngine.unmarshal(getREGISTRY_CENTER().get("/" + getJobName() + "/config"), JobConfigurationPOJO.class).toJobConfiguration();
+        assertThat(jobConfig.getShardingTotalCount(), is(3));
+        assertThat(jobConfig.getCron(), is("*/10 * * * * ?"));
+        assertNull(jobConfig.getTimeZone());
+        assertThat(jobConfig.getShardingItemParameters(), is("0=a,1=b,2=c"));
+        assertThat(getREGISTRY_CENTER().get("/" + getJobName() + "/servers/" + JobRegistry.getInstance().getJobInstance(getJobName()).getServerIp()), is(ServerStatus.ENABLED.name()));
+        assertThat(getREGISTRY_CENTER().get("/" + getJobName() + "/leader/election/instance"), is(JobRegistry.getInstance().getJobInstance(getJobName()).getJobInstanceId()));
+        assertTrue(getREGISTRY_CENTER().isExisted("/" + getJobName() + "/instances/" + JobRegistry.getInstance().getJobInstance(getJobName()).getJobInstanceId()));
+        getREGISTRY_CENTER().remove("/" + getJobName() + "/leader/election");
+        assertTrue(getLeaderService().isLeaderUntilBlock());
+    }
+    
+    @Test
+    public void assertJobInit() {
+        while (!((AnnotationSimpleJob) getElasticJob()).isCompleted()) {
+            BlockUtils.waitingShortTime();
+        }
+        assertTrue(getREGISTRY_CENTER().isExisted("/" + getJobName() + "/sharding"));
+    }
+    
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/ElasticJobSpringBootScannerTest.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/ElasticJobSpringBootScannerTest.java
new file mode 100644
index 000000000..25b948701
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/ElasticJobSpringBootScannerTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.spring.boot.job;
+
+import org.apache.shardingsphere.elasticjob.infra.concurrent.BlockUtils;
+import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.ScheduleJobBootstrap;
+import org.apache.shardingsphere.elasticjob.lite.spring.boot.job.fixture.EmbedTestingServer;
+import org.apache.shardingsphere.elasticjob.lite.spring.boot.job.fixture.job.impl.AnnotationCustomJob;
+import org.apache.shardingsphere.elasticjob.lite.spring.core.scanner.ElasticJobScan;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.test.context.ActiveProfiles;
+import org.springframework.test.context.junit4.AbstractJUnit4SpringContextTests;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+@SpringBootTest
+@ActiveProfiles("elasticjob")
+@ElasticJobScan(basePackages = "org.apache.shardingsphere.elasticjob.lite.spring.boot.job.fixture.job.impl")
+public class ElasticJobSpringBootScannerTest extends AbstractJUnit4SpringContextTests {
+    
+    @BeforeClass
+    public static void init() {
+        EmbedTestingServer.start();
+        AnnotationCustomJob.reset();
+    }
+    
+    @Test
+    public void assertDefaultBeanNameWithTypeJob() {
+        while (!AnnotationCustomJob.isCompleted()) {
+            BlockUtils.waitingShortTime();
+        }
+        assertTrue(AnnotationCustomJob.isCompleted());
+        assertNotNull(applicationContext);
+        assertNotNull(applicationContext.getBean("annotationCustomJobSchedule", ScheduleJobBootstrap.class));
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/fixture/job/impl/AnnotationCustomJob.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/fixture/job/impl/AnnotationCustomJob.java
new file mode 100644
index 000000000..dfaf75a85
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-boot-starter/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/boot/job/fixture/job/impl/AnnotationCustomJob.java
@@ -0,0 +1,58 @@
+/*
+ * 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.spring.boot.job.fixture.job.impl;
+
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobProp;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+import org.apache.shardingsphere.elasticjob.lite.spring.boot.job.fixture.job.CustomJob;
+import org.springframework.transaction.annotation.Transactional;
+
+@Slf4j
+@Transactional(rollbackFor = Exception.class)
+@ElasticJobConfiguration(
+        jobName = "annotationCustomJobSchedule",
+        description = "desc",
+        shardingTotalCount = 3,
+        shardingItemParameters = "0=a,1=b,2=c",
+        cron = "*/1 * * * * ?",
+        props = {
+                @ElasticJobProp(key = "print.title", value = "test title"),
+                @ElasticJobProp(key = "print.content", value = "test content")
+        }
+)
+public class AnnotationCustomJob implements CustomJob {
+
+    @Getter
+    private static volatile boolean completed;
+
+    @Override
+    public void execute(final ShardingContext shardingContext) {
+        log.info("AnnotationCustomJob execut");
+        completed = true;
+    }
+
+    /**
+     * Set completed to false.
+     */
+    public static void reset() {
+        completed = false;
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ClassPathJobScanner.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ClassPathJobScanner.java
new file mode 100644
index 000000000..c8028ad09
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ClassPathJobScanner.java
@@ -0,0 +1,85 @@
+/*
+ * 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.spring.core.scanner;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.lite.api.bootstrap.impl.ScheduleJobBootstrap;
+import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
+import org.springframework.beans.factory.config.BeanDefinitionHolder;
+import org.springframework.beans.factory.config.RuntimeBeanReference;
+import org.springframework.beans.factory.support.BeanDefinitionBuilder;
+import org.springframework.beans.factory.support.BeanDefinitionRegistry;
+import org.springframework.context.annotation.ClassPathBeanDefinitionScanner;
+import org.springframework.context.annotation.ScannedGenericBeanDefinition;
+import org.springframework.core.type.filter.AnnotationTypeFilter;
+
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * A {@link ClassPathBeanDefinitionScanner} that registers ScheduleJobBootstrap by {@code basePackage}.
+ *
+ * @see ScheduleJobBootstrap
+ */
+public class ClassPathJobScanner extends ClassPathBeanDefinitionScanner {
+
+    public ClassPathJobScanner(final BeanDefinitionRegistry registry) {
+        super(registry, false);
+    }
+
+    /**
+     * Calls the parent search that will search and register all the candidates by {@code ElasticJobConfiguration}.
+     *
+     * @param basePackages the packages to check for annotated classes
+     */
+    @Override
+    protected Set<BeanDefinitionHolder> doScan(final String... basePackages) {
+        addIncludeFilter(new AnnotationTypeFilter(ElasticJobConfiguration.class));
+        Set<BeanDefinitionHolder> beanDefinitions = super.doScan(basePackages);
+        if (!beanDefinitions.isEmpty()) {
+            processBeanDefinitions(beanDefinitions);
+        }
+        return beanDefinitions;
+    }
+
+    private void processBeanDefinitions(final Set<BeanDefinitionHolder> beanDefinitions) {
+        BeanDefinitionRegistry registry = getRegistry();
+        for (BeanDefinitionHolder holder : beanDefinitions) {
+            ScannedGenericBeanDefinition definition = (ScannedGenericBeanDefinition) holder.getBeanDefinition();
+            Class<?> jobClass;
+            try {
+                jobClass = Class.forName(definition.getMetadata().getClassName());
+            } catch (ClassNotFoundException ex) {
+                //TODO: log
+                continue;
+            }
+            ElasticJobConfiguration jobAnnotation = jobClass.getAnnotation(ElasticJobConfiguration.class);
+            String registryCenter = jobAnnotation.registryCenter();
+            BeanDefinitionBuilder factory = BeanDefinitionBuilder.rootBeanDefinition(ScheduleJobBootstrap.class);
+            factory.setInitMethodName("schedule");
+            if (!StringUtils.isEmpty(registryCenter)) {
+                factory.addConstructorArgReference(registryCenter);
+            } else {
+                factory.addConstructorArgValue(new RuntimeBeanReference(CoordinatorRegistryCenter.class));
+            }
+            factory.addConstructorArgReference(Objects.requireNonNull(holder.getBeanName()));
+            registry.registerBeanDefinition(jobAnnotation.jobName(), factory.getBeanDefinition());
+        }
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScan.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScan.java
new file mode 100644
index 000000000..8e9ad9598
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScan.java
@@ -0,0 +1,51 @@
+/*
+ * 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.spring.core.scanner;
+
+import org.springframework.context.annotation.Import;
+
+import java.lang.annotation.Documented;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Use this annotation to register Elastic job.
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+@Import(ElasticJobScanRegistrar.class)
+@Target(ElementType.TYPE)
+public @interface ElasticJobScan {
+    
+    /**
+     * Alias for the {@link #basePackages()} attribute.
+     *
+     * @return Base packages name
+     */
+    String[] value() default "";
+
+    /**
+     * Base packages to scan for Elastic job.
+     *
+     * @return Base packages name
+     */
+    String[] basePackages() default {};
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScanRegistrar.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScanRegistrar.java
new file mode 100644
index 000000000..5183bae7f
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/ElasticJobScanRegistrar.java
@@ -0,0 +1,65 @@
+/*
+ * 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.spring.core.scanner;
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder;
+import org.springframework.beans.factory.support.BeanDefinitionRegistry;
+import org.springframework.context.annotation.ImportBeanDefinitionRegistrar;
+import org.springframework.core.annotation.AnnotationAttributes;
+import org.springframework.core.type.AnnotationMetadata;
+import org.springframework.util.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * A {@link ImportBeanDefinitionRegistrar} to allow annotation configuration of Elastic Job scanning.
+ *
+ * @see ClassPathJobScanner
+ */
+public class ElasticJobScanRegistrar implements ImportBeanDefinitionRegistrar {
+
+    @Override
+    public void registerBeanDefinitions(final AnnotationMetadata importingClassMetadata,
+                                        final BeanDefinitionRegistry registry) {
+        AnnotationAttributes elasticJobScanAttrs =
+                AnnotationAttributes.fromMap(importingClassMetadata.getAnnotationAttributes(ElasticJobScan.class.getName()));
+        if (elasticJobScanAttrs != null) {
+            registerBeanDefinitions(importingClassMetadata, elasticJobScanAttrs, registry);
+        }
+    }
+
+    private void registerBeanDefinitions(final AnnotationMetadata annoMeta, final AnnotationAttributes annoAttrs,
+            final BeanDefinitionRegistry registry) {
+        BeanDefinitionBuilder factory = BeanDefinitionBuilder.rootBeanDefinition(JobScannerConfiguration.class);
+
+        List<String> basePackages = new ArrayList<>();
+        basePackages.addAll(Arrays.stream(annoAttrs.getStringArray("value")).filter(StringUtils::hasText)
+                .collect(Collectors.toList()));
+        basePackages.addAll(Arrays.stream(annoAttrs.getStringArray("basePackages")).filter(StringUtils::hasText)
+                .collect(Collectors.toList()));
+        factory.addConstructorArgValue(basePackages);
+        registry.registerBeanDefinition(generateBaseBeanName(annoMeta), factory.getBeanDefinition());
+    }
+
+    private static String generateBaseBeanName(final AnnotationMetadata importingClassMetadata) {
+        return importingClassMetadata.getClassName() + "#" + ElasticJobScanRegistrar.class.getSimpleName();
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/JobScannerConfiguration.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/JobScannerConfiguration.java
new file mode 100644
index 000000000..9cd890da4
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-core/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/core/scanner/JobScannerConfiguration.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spring.core.scanner;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.factory.InitializingBean;
+import org.springframework.beans.factory.config.ConfigurableListableBeanFactory;
+import org.springframework.beans.factory.support.BeanDefinitionRegistry;
+import org.springframework.beans.factory.support.BeanDefinitionRegistryPostProcessor;
+import org.springframework.util.Assert;
+
+/**
+ * BeanDefinitionRegistryPostProcessor that searches recursively starting from a base package for interfaces.
+ *
+ */
+@Getter
+@RequiredArgsConstructor
+public class JobScannerConfiguration implements BeanDefinitionRegistryPostProcessor, InitializingBean {
+
+    private final String[] basePackages;
+
+    @Override
+    public void afterPropertiesSet() {
+        Assert.notNull(this.basePackages, "Property 'basePackage' is required");
+    }
+
+    @Override
+    public void postProcessBeanFactory(final ConfigurableListableBeanFactory beanFactory) throws BeansException {
+        // left intentionally blank
+    }
+
+    @Override
+    public void postProcessBeanDefinitionRegistry(final BeanDefinitionRegistry registry) throws BeansException {
+        ClassPathJobScanner classPathJobScanner = new ClassPathJobScanner(registry);
+        classPathJobScanner.scan(basePackages);
+    }
+
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/ElasticJobNamespaceHandler.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/ElasticJobNamespaceHandler.java
index 374746bf5..d57ba6205 100644
--- a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/ElasticJobNamespaceHandler.java
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/ElasticJobNamespaceHandler.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.elasticjob.lite.spring.namespace;
 
+import org.apache.shardingsphere.elasticjob.lite.spring.namespace.scanner.parser.JobScannerBeanDefinitionParser;
 import org.apache.shardingsphere.elasticjob.lite.spring.namespace.job.parser.JobBeanDefinitionParser;
 import org.apache.shardingsphere.elasticjob.lite.spring.namespace.reg.parser.ZookeeperBeanDefinitionParser;
 import org.apache.shardingsphere.elasticjob.lite.spring.namespace.snapshot.parser.SnapshotBeanDefinitionParser;
@@ -34,5 +35,6 @@ public final class ElasticJobNamespaceHandler extends NamespaceHandlerSupport {
         registerBeanDefinitionParser("zookeeper", new ZookeeperBeanDefinitionParser());
         registerBeanDefinitionParser("snapshot", new SnapshotBeanDefinitionParser());
         registerBeanDefinitionParser("rdb-tracing", new TracingBeanDefinitionParser());
+        registerBeanDefinitionParser("job-scanner", new JobScannerBeanDefinitionParser());
     }
 }
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/parser/JobScannerBeanDefinitionParser.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/parser/JobScannerBeanDefinitionParser.java
new file mode 100644
index 000000000..f7359d63a
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/parser/JobScannerBeanDefinitionParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spring.namespace.scanner.parser;
+
+import org.apache.shardingsphere.elasticjob.lite.spring.core.scanner.JobScannerConfiguration;
+import org.apache.shardingsphere.elasticjob.lite.spring.namespace.scanner.tag.JobScannerBeanDefinitionTag;
+import org.springframework.beans.factory.support.AbstractBeanDefinition;
+import org.springframework.beans.factory.support.BeanDefinitionBuilder;
+import org.springframework.beans.factory.xml.AbstractBeanDefinitionParser;
+import org.springframework.beans.factory.xml.ParserContext;
+import org.w3c.dom.Element;
+
+/**
+ * Job scanner bean definition parser.
+ */
+public final class JobScannerBeanDefinitionParser extends AbstractBeanDefinitionParser {
+    
+    @Override
+    protected AbstractBeanDefinition parseInternal(final Element element, final ParserContext parserContext) {
+        BeanDefinitionBuilder factory = BeanDefinitionBuilder.rootBeanDefinition(JobScannerConfiguration.class);
+        String attribute = element.getAttribute(JobScannerBeanDefinitionTag.BASE_PACKAGE);
+        factory.addConstructorArgValue(attribute);
+        return factory.getBeanDefinition();
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/tag/JobScannerBeanDefinitionTag.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/tag/JobScannerBeanDefinitionTag.java
new file mode 100644
index 000000000..859756eea
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/tag/JobScannerBeanDefinitionTag.java
@@ -0,0 +1,29 @@
+/*
+ * 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.spring.namespace.scanner.tag;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+/**
+ * Job scanner bean definition tag.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class JobScannerBeanDefinitionTag {
+    public static final String BASE_PACKAGE = "base-package";
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/resources/META-INF/namespace/elasticjob.xsd b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/resources/META-INF/namespace/elasticjob.xsd
index ef91aba38..308a8e460 100644
--- a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/resources/META-INF/namespace/elasticjob.xsd
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/main/resources/META-INF/namespace/elasticjob.xsd
@@ -21,8 +21,19 @@
             xmlns:beans="http://www.springframework.org/schema/beans"
             targetNamespace="http://shardingsphere.apache.org/schema/elasticjob"
             elementFormDefault="qualified">
-    <xsd:import namespace="http://www.springframework.org/schema/beans" schemaLocation="http://www.springframework.org/schema/beans/spring-beans.xsd" />
-    
+    <xsd:import namespace="http://www.springframework.org/schema/beans"
+                schemaLocation="http://www.springframework.org/schema/beans/spring-beans.xsd"/>
+
+    <xsd:element name="job-scanner">
+        <xsd:complexType>
+            <xsd:complexContent>
+                <xsd:extension base="beans:identifiedType">
+                    <xsd:attribute name="base-package" type="xsd:string" use="required"/>
+                </xsd:extension>
+            </xsd:complexContent>
+        </xsd:complexType>
+    </xsd:element>
+
     <xsd:element name="job">
         <xsd:complexType>
             <xsd:complexContent>
@@ -57,7 +68,7 @@
             </xsd:complexContent>
         </xsd:complexType>
     </xsd:element>
-    
+
     <xsd:element name="zookeeper">
         <xsd:complexType>
             <xsd:complexContent>
@@ -74,7 +85,7 @@
             </xsd:complexContent>
         </xsd:complexType>
     </xsd:element>
-    
+
     <xsd:element name="rdb-tracing">
         <xsd:complexType>
             <xsd:complexContent>
@@ -84,7 +95,7 @@
             </xsd:complexContent>
         </xsd:complexType>
     </xsd:element>
-    
+
     <xsd:element name="snapshot">
         <xsd:complexType>
             <xsd:complexContent>
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/job/annotation/AnnotationSimpleJob.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/job/annotation/AnnotationSimpleJob.java
new file mode 100644
index 000000000..28da7b815
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/fixture/job/annotation/AnnotationSimpleJob.java
@@ -0,0 +1,55 @@
+/*
+ * 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.spring.namespace.fixture.job.annotation;
+
+import lombok.Getter;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobConfiguration;
+import org.apache.shardingsphere.elasticjob.annotation.ElasticJobProp;
+import org.apache.shardingsphere.elasticjob.api.ShardingContext;
+import org.apache.shardingsphere.elasticjob.simple.job.SimpleJob;
+
+@Getter
+@ElasticJobConfiguration(
+    jobName = "simpleJob",
+    registryCenter = "regCenter",
+    description = "desc",
+    shardingTotalCount = 3,
+    shardingItemParameters = "0=a,1=b,2=c",
+    cron = "*/1 * * * * ?",
+    props = {
+        @ElasticJobProp(key = "print.title", value = "test title"),
+        @ElasticJobProp(key = "print.content", value = "test content")
+    }
+)
+public final class AnnotationSimpleJob implements SimpleJob {
+
+    @Getter
+    private static volatile boolean completed;
+    
+    @Override
+    public void execute(final ShardingContext shardingContext) {
+        completed = true;
+    }
+
+    /**
+     * Set completed to false.
+     */
+    public static void reset() {
+        completed = false;
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/AbstractJobSpringIntegrateTest.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/AbstractJobSpringIntegrateTest.java
new file mode 100644
index 000000000..5cc4ffdf2
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/AbstractJobSpringIntegrateTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.spring.namespace.scanner;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.elasticjob.infra.concurrent.BlockUtils;
+import org.apache.shardingsphere.elasticjob.lite.internal.schedule.JobRegistry;
+import org.apache.shardingsphere.elasticjob.lite.spring.namespace.fixture.job.annotation.AnnotationSimpleJob;
+import org.apache.shardingsphere.elasticjob.lite.spring.namespace.test.AbstractZookeeperJUnit4SpringContextTests;
+import org.apache.shardingsphere.elasticjob.reg.base.CoordinatorRegistryCenter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+
+import static org.junit.Assert.assertTrue;
+
+@RequiredArgsConstructor
+public abstract class AbstractJobSpringIntegrateTest extends AbstractZookeeperJUnit4SpringContextTests {
+    
+    private final String simpleJobName;
+
+    @Autowired
+    private CoordinatorRegistryCenter regCenter;
+    
+    @Before
+    @After
+    public void reset() {
+        AnnotationSimpleJob.reset();
+    }
+    
+    @After
+    public void tearDown() {
+        JobRegistry.getInstance().shutdown(simpleJobName);
+    }
+    
+    @Test
+    public void assertSpringJobBean() {
+        assertSimpleElasticJobBean();
+    }
+    
+    private void assertSimpleElasticJobBean() {
+        while (!AnnotationSimpleJob.isCompleted()) {
+            BlockUtils.waitingShortTime();
+        }
+        assertTrue(AnnotationSimpleJob.isCompleted());
+        assertTrue(regCenter.isExisted("/" + simpleJobName + "/sharding"));
+    }
+    
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/JobScannerTest.java b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/JobScannerTest.java
new file mode 100644
index 000000000..d61d106bf
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/java/org/apache/shardingsphere/elasticjob/lite/spring/namespace/scanner/JobScannerTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.spring.namespace.scanner;
+
+import org.springframework.test.context.ContextConfiguration;
+
+@ContextConfiguration(locations = "classpath:META-INF/scanner/jobScannerContext.xml")
+public final class JobScannerTest extends AbstractJobSpringIntegrateTest {
+    
+    public JobScannerTest() {
+        super("simpleJob");
+    }
+}
diff --git a/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/resources/META-INF/scanner/jobScannerContext.xml b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/resources/META-INF/scanner/jobScannerContext.xml
new file mode 100644
index 000000000..fde581aaa
--- /dev/null
+++ b/elasticjob-lite/elasticjob-lite-spring/elasticjob-lite-spring-namespace/src/test/resources/META-INF/scanner/jobScannerContext.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xmlns:elasticjob="http://shardingsphere.apache.org/schema/elasticjob"
+    xsi:schemaLocation="http://www.springframework.org/schema/beans 
+                        http://www.springframework.org/schema/beans/spring-beans.xsd 
+                        http://www.springframework.org/schema/context 
+                        http://www.springframework.org/schema/context/spring-context.xsd 
+                        http://shardingsphere.apache.org/schema/elasticjob
+                        http://shardingsphere.apache.org/schema/elasticjob/elasticjob.xsd
+                        ">
+    <context:property-placeholder location="classpath:conf/job/conf.properties" />
+
+    <elasticjob:zookeeper id="regCenter" server-lists="${regCenter.serverLists}" namespace="${regCenter.namespace}" base-sleep-time-milliseconds="${regCenter.baseSleepTimeMilliseconds}"
+                          max-sleep-time-milliseconds="${regCenter.maxSleepTimeMilliseconds}" max-retries="${regCenter.maxRetries}" />
+
+    <elasticjob:job-scanner id="job-scanner" base-package="org.apache.shardingsphere.elasticjob.lite.spring.namespace.fixture.job.annotation"/>
+</beans>