diff --git a/pom.xml b/pom.xml
index eb01fbdcc5..348c8d6771 100644
--- a/pom.xml
+++ b/pom.xml
@@ -152,6 +152,26 @@
0.0.39
+
+
+
+ org.slf4j
+ jcl-over-slf4j
+ ${slf4j.version}
+
+
+ org.slf4j
+ slf4j-api
+ ${slf4j.version}
+
+
+ org.slf4j
+ slf4j-simple
+ ${slf4j.version}
+
+
+
+
diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/AbstractJob.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/AbstractJob.java
index 34d6d19f58..83efa13f59 100644
--- a/spring-batch-core/src/main/java/org/springframework/batch/core/job/AbstractJob.java
+++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/AbstractJob.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -17,8 +17,7 @@
package org.springframework.batch.core.job;
import java.time.LocalDateTime;
-import java.util.Collection;
-import java.util.List;
+import java.util.*;
import java.util.stream.Collectors;
import io.micrometer.core.instrument.LongTaskTimer;
@@ -43,6 +42,7 @@
import org.springframework.batch.core.StartLimitExceededException;
import org.springframework.batch.core.Step;
import org.springframework.batch.core.StepExecution;
+import org.springframework.batch.core.job.builder.AlreadyUsedStepNameException;
import org.springframework.batch.core.launch.NoSuchJobException;
import org.springframework.batch.core.launch.support.ExitCodeMapper;
import org.springframework.batch.core.listener.CompositeJobExecutionListener;
@@ -300,6 +300,7 @@ public final void execute(JobExecution execution) {
execution.setStartTime(LocalDateTime.now());
updateStatus(execution, BatchStatus.STARTED);
+ checkStepNamesUnicity();
listener.beforeJob(execution);
@@ -368,11 +369,11 @@ public final void execute(JobExecution execution) {
finally {
JobSynchronizationManager.release();
}
-
}
-
}
+ protected abstract void checkStepNamesUnicity() throws AlreadyUsedStepNameException;
+
private void stopObservation(JobExecution execution, Observation observation) {
List throwables = execution.getFailureExceptions();
if (!throwables.isEmpty()) {
@@ -430,6 +431,16 @@ else if (ex instanceof NoSuchJobException || ex.getCause() instanceof NoSuchJobE
return exitStatus;
}
+ protected static void addToMapCheckingUnicity(Map map, Step step, String name)
+ throws AlreadyUsedStepNameException {
+ map.merge(name, step, (old, value) -> {
+ if (!old.equals(value)) {
+ throw new AlreadyUsedStepNameException(name);
+ }
+ return old;
+ });
+ }
+
private void updateStatus(JobExecution jobExecution, BatchStatus status) {
jobExecution.setStatus(status);
jobRepository.update(jobExecution);
diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/SimpleJob.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/SimpleJob.java
index b22317ef28..e679447f02 100644
--- a/spring-batch-core/src/main/java/org/springframework/batch/core/job/SimpleJob.java
+++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/SimpleJob.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -16,9 +16,7 @@
package org.springframework.batch.core.job;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
+import java.util.*;
import org.springframework.batch.core.BatchStatus;
import org.springframework.batch.core.Job;
@@ -145,4 +143,9 @@ protected void doExecute(JobExecution execution)
}
}
+ @Override
+ protected void checkStepNamesUnicity() {
+ // noop : steps of SimpleJob can share the same name
+ }
+
}
diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/builder/AlreadyUsedStepNameException.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/builder/AlreadyUsedStepNameException.java
new file mode 100644
index 0000000000..4547447e46
--- /dev/null
+++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/builder/AlreadyUsedStepNameException.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2006-2025 the original author or authors.
+ *
+ * Licensed 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
+ *
+ * https://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.springframework.batch.core.job.builder;
+
+/**
+ * Exception to indicate the name of a step is already used by a different step in the
+ * same flow. Step names must be unique within a flow definition because the search of the
+ * next step to find relies on the step name
+ *
+ * @author Fabrice Bibonne
+ */
+public class AlreadyUsedStepNameException extends RuntimeException {
+
+ public AlreadyUsedStepNameException(String name) {
+ super("the name " + name + " is already used");
+ }
+
+}
diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/flow/FlowJob.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/flow/FlowJob.java
index 33e2f491fe..adaf877112 100644
--- a/spring-batch-core/src/main/java/org/springframework/batch/core/job/flow/FlowJob.java
+++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/flow/FlowJob.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -15,19 +15,20 @@
*/
package org.springframework.batch.core.job.flow;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
import org.springframework.batch.core.Job;
import org.springframework.batch.core.JobExecution;
import org.springframework.batch.core.JobExecutionException;
import org.springframework.batch.core.Step;
import org.springframework.batch.core.job.AbstractJob;
import org.springframework.batch.core.job.SimpleStepHandler;
+import org.springframework.batch.core.job.builder.AlreadyUsedStepNameException;
import org.springframework.batch.core.step.StepHolder;
import org.springframework.batch.core.step.StepLocator;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
/**
* Implementation of the {@link Job} interface that allows for complex flows of steps,
* rather than requiring sequential execution. In general, this job implementation was
@@ -74,9 +75,7 @@ public void setFlow(Flow flow) {
*/
@Override
public Step getStep(String stepName) {
- if (!initialized) {
- init();
- }
+ init();
return stepMap.get(stepName);
}
@@ -84,30 +83,34 @@ public Step getStep(String stepName) {
* Initialize the step names
*/
private void init() {
- findSteps(flow, stepMap);
- initialized = true;
+ if (!initialized) {
+ findStepsThrowingIfNameNotUnique(flow);
+ initialized = true;
+ }
}
- private void findSteps(Flow flow, Map map) {
+ private void findStepsThrowingIfNameNotUnique(Flow flow) {
for (State state : flow.getStates()) {
if (state instanceof StepLocator locator) {
for (String name : locator.getStepNames()) {
- map.put(name, locator.getStep(name));
+ addToMapCheckingUnicity(this.stepMap, locator.getStep(name), name);
}
}
- else if (state instanceof StepHolder) {
- Step step = ((StepHolder) state).getStep();
- String name = step.getName();
- stepMap.put(name, step);
+ // TODO remove this else bock ? not executed during tests : the only State
+ // which implements StepHolder is StepState which already implements
+ // StepLocator
+ // within tests coverage `state instanceof StepHolder` is false 30 times/30
+ else if (state instanceof StepHolder stepHolder) {
+ Step step = stepHolder.getStep();
+ addToMapCheckingUnicity(this.stepMap, step, step.getName());
}
- else if (state instanceof FlowHolder) {
- for (Flow subflow : ((FlowHolder) state).getFlows()) {
- findSteps(subflow, map);
+ else if (state instanceof FlowHolder flowHolder) {
+ for (Flow subflow : flowHolder.getFlows()) {
+ findStepsThrowingIfNameNotUnique(subflow);
}
}
}
-
}
/**
@@ -115,9 +118,7 @@ else if (state instanceof FlowHolder) {
*/
@Override
public Collection getStepNames() {
- if (!initialized) {
- init();
- }
+ init();
return stepMap.keySet();
}
@@ -139,4 +140,9 @@ protected void doExecute(final JobExecution execution) throws JobExecutionExcept
}
}
+ @Override
+ protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
+ init();
+ }
+
}
diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/job/ExtendedAbstractJobTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/job/ExtendedAbstractJobTests.java
index 79a5684b1f..71ca114d96 100644
--- a/spring-batch-core/src/test/java/org/springframework/batch/core/job/ExtendedAbstractJobTests.java
+++ b/spring-batch-core/src/test/java/org/springframework/batch/core/job/ExtendedAbstractJobTests.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -17,31 +17,20 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
-import org.springframework.batch.core.BatchStatus;
-import org.springframework.batch.core.JobExecution;
-import org.springframework.batch.core.JobExecutionException;
-import org.springframework.batch.core.JobInterruptedException;
-import org.springframework.batch.core.JobParameters;
-import org.springframework.batch.core.JobParametersInvalidException;
-import org.springframework.batch.core.Step;
-import org.springframework.batch.core.StepExecution;
+import org.springframework.batch.core.*;
import org.springframework.batch.core.repository.JobRepository;
import org.springframework.batch.core.repository.support.JobRepositoryFactoryBean;
import org.springframework.batch.core.step.StepSupport;
-import org.springframework.jdbc.support.JdbcTransactionManager;
import org.springframework.jdbc.datasource.embedded.EmbeddedDatabase;
import org.springframework.jdbc.datasource.embedded.EmbeddedDatabaseBuilder;
+import org.springframework.jdbc.support.JdbcTransactionManager;
import org.springframework.lang.Nullable;
import java.time.LocalDateTime;
import java.util.Collection;
import java.util.Collections;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.*;
/**
* @author Dave Syer
@@ -215,6 +204,10 @@ public StubJob() {
protected void doExecute(JobExecution execution) throws JobExecutionException {
}
+ @Override
+ protected void checkStepNamesUnicity() {
+ }
+
@Override
public Step getStep(String stepName) {
return null;
diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/job/SimpleJobTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/job/SimpleJobTests.java
index c850e8c77f..0219609d1e 100644
--- a/spring-batch-core/src/test/java/org/springframework/batch/core/job/SimpleJobTests.java
+++ b/spring-batch-core/src/test/java/org/springframework/batch/core/job/SimpleJobTests.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -513,6 +513,25 @@ void testGetMultipleJobParameters() throws Exception {
}
+ @Test
+ public void testMultipleStepsWithSameName() {
+ job.setName("MultipleStepsWithSameName");
+ String sharedName = "stepName";
+ final List executionsCallbacks = new ArrayList<>();
+ StubStep sharedNameStep1 = new StubStep(sharedName, jobRepository);
+ sharedNameStep1.setCallback(() -> executionsCallbacks.add("step1"));
+ job.addStep(sharedNameStep1);
+ StubStep sharedNameStep2 = new StubStep(sharedName, jobRepository);
+ sharedNameStep2.setCallback(() -> executionsCallbacks.add("step2"));
+ job.addStep(sharedNameStep2);
+ StubStep sharedNameStep3 = new StubStep(sharedName, jobRepository);
+ sharedNameStep3.setCallback(() -> executionsCallbacks.add("step3"));
+ job.addStep(sharedNameStep3);
+ job.execute(jobExecution);
+ assertEquals(List.of("step1", "step2", "step3"), executionsCallbacks);
+ assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus());
+ }
+
/*
* Check JobRepository to ensure status is being saved.
*/
diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/job/builder/FlowJobBuilderTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/job/builder/FlowJobBuilderTests.java
index 909c04c0d1..d63e905e98 100644
--- a/spring-batch-core/src/test/java/org/springframework/batch/core/job/builder/FlowJobBuilderTests.java
+++ b/spring-batch-core/src/test/java/org/springframework/batch/core/job/builder/FlowJobBuilderTests.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2024 the original author or authors.
+ * Copyright 2012-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -15,27 +15,12 @@
*/
package org.springframework.batch.core.job.builder;
-import java.util.Arrays;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import javax.sql.DataSource;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
-import org.springframework.batch.core.BatchStatus;
-import org.springframework.batch.core.ExitStatus;
-import org.springframework.batch.core.Job;
-import org.springframework.batch.core.JobExecution;
-import org.springframework.batch.core.JobInterruptedException;
-import org.springframework.batch.core.JobParameters;
-import org.springframework.batch.core.JobParametersBuilder;
-import org.springframework.batch.core.Step;
-import org.springframework.batch.core.StepExecution;
-import org.springframework.batch.core.UnexpectedJobExecutionException;
+import org.springframework.batch.core.*;
import org.springframework.batch.core.configuration.annotation.EnableBatchProcessing;
import org.springframework.batch.core.configuration.annotation.JobScope;
import org.springframework.batch.core.job.flow.Flow;
@@ -43,29 +28,41 @@
import org.springframework.batch.core.job.flow.JobExecutionDecider;
import org.springframework.batch.core.job.flow.support.SimpleFlow;
import org.springframework.batch.core.launch.JobLauncher;
+import org.springframework.batch.core.repository.JobExecutionAlreadyRunningException;
+import org.springframework.batch.core.repository.JobInstanceAlreadyCompleteException;
import org.springframework.batch.core.repository.JobRepository;
+import org.springframework.batch.core.repository.JobRestartException;
import org.springframework.batch.core.repository.support.JobRepositoryFactoryBean;
+import org.springframework.batch.core.scope.context.ChunkContext;
import org.springframework.batch.core.step.StepSupport;
import org.springframework.batch.core.step.builder.StepBuilder;
import org.springframework.batch.item.support.ListItemReader;
import org.springframework.batch.repeat.RepeatStatus;
import org.springframework.batch.support.transaction.ResourcelessTransactionManager;
+import org.springframework.beans.factory.annotation.Qualifier;
import org.springframework.beans.factory.annotation.Value;
import org.springframework.context.ApplicationContext;
import org.springframework.context.annotation.AnnotationConfigApplicationContext;
import org.springframework.context.annotation.Bean;
import org.springframework.context.annotation.Configuration;
import org.springframework.core.task.SimpleAsyncTaskExecutor;
-import org.springframework.jdbc.support.JdbcTransactionManager;
import org.springframework.jdbc.datasource.embedded.EmbeddedDatabase;
import org.springframework.jdbc.datasource.embedded.EmbeddedDatabaseBuilder;
+import org.springframework.jdbc.support.JdbcTransactionManager;
import org.springframework.lang.Nullable;
import org.springframework.transaction.PlatformTransactionManager;
+import javax.sql.DataSource;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
/**
* @author Dave Syer
* @author Mahmoud Ben Hassine
- *
+ * @author Fabrice Bibonne
*/
class FlowJobBuilderTests {
@@ -267,26 +264,6 @@ public FlowExecutionStatus decide(JobExecution jobExecution, @Nullable StepExecu
assertEquals(1, execution.getStepExecutions().size());
}
- @Test
- void testBuildWithDeciderPriorityOnWildcardCount() {
- JobExecutionDecider decider = (jobExecution, stepExecution) -> new FlowExecutionStatus("COMPLETED_PARTIALLY");
- JobFlowBuilder builder = new JobBuilder("flow_priority", jobRepository).start(decider);
- builder.on("**").end();
- builder.on("*").fail();
- builder.build().preventRestart().build().execute(execution);
- assertEquals(BatchStatus.COMPLETED, execution.getStatus());
- }
-
- @Test
- void testBuildWithDeciderPriorityWithEqualWildcard() {
- JobExecutionDecider decider = (jobExecution, stepExecution) -> new FlowExecutionStatus("COMPLETED_PARTIALLY");
- JobFlowBuilder builder = new JobBuilder("flow_priority", jobRepository).start(decider);
- builder.on("COMPLETED*").end();
- builder.on("*").fail();
- builder.build().preventRestart().build().execute(execution);
- assertEquals(BatchStatus.COMPLETED, execution.getStatus());
- }
-
@Test
void testBuildWithDeciderPriority() {
JobExecutionDecider decider = (jobExecution, stepExecution) -> new FlowExecutionStatus("COMPLETED_PARTIALLY");
@@ -388,6 +365,121 @@ void testBuildWithJobScopedStep() throws Exception {
assertEquals(ExitStatus.COMPLETED, jobExecution.getExitStatus());
}
+ // https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
+ @Test
+ void testStepNamesMustBeUniqueWithinFlowDefinition() throws JobInstanceAlreadyCompleteException,
+ JobExecutionAlreadyRunningException, JobParametersInvalidException, JobRestartException {
+ ApplicationContext context = new AnnotationConfigApplicationContext(JobConfigurationForStepNameUnique.class);
+ JobLauncher jobLauncher = context.getBean(JobLauncher.class);
+ Job job = context.getBean(Job.class);
+ JobExecution jobExecution = jobLauncher.run(job,
+ new JobParametersBuilder().addLong("random", 2L)
+ .addString("stepTwo.name", JobConfigurationForStepNameUnique.SHARED_NAME)
+ .toJobParameters());
+ Assertions.assertTrue(jobExecution.getAllFailureExceptions()
+ .stream()
+ .map(Object::getClass)
+ .anyMatch(AlreadyUsedStepNameException.class::equals));
+ assertEquals(ExitStatus.FAILED.getExitCode(), jobExecution.getExitStatus().getExitCode());
+ jobExecution = jobLauncher.run(job,
+ new JobParametersBuilder().addLong("random", 1L)
+ .addString("stepTwo.name", JobConfigurationForStepNameUnique.SHARED_NAME)
+ .toJobParameters());
+ Assertions.assertTrue(jobExecution.getAllFailureExceptions()
+ .stream()
+ .map(Object::getClass)
+ .anyMatch(AlreadyUsedStepNameException.class::equals));
+ assertEquals(ExitStatus.FAILED.getExitCode(), jobExecution.getExitStatus().getExitCode());
+ }
+
+ @EnableBatchProcessing
+ @Configuration
+ static class JobConfigurationForStepNameUnique {
+
+ private static final String SHARED_NAME = "sharedName";
+
+ private static final Log logger = LogFactory.getLog(FlowJobBuilderTests.class);
+
+ @Bean
+ @JobScope
+ public Step conditionalStep(JobRepository jobRepository, PlatformTransactionManager transactionManager,
+ @Value("#{jobParameters['random']}") Integer random) {
+ return new StepBuilder("conditionalStep", jobRepository)
+ .tasklet((StepContribution contribution, ChunkContext chunkContext) -> {
+ String exitStatus = (random % 2 == 0) ? "EVEN" : "ODD";
+ logger.info("'conditionalStep' with exitStatus " + exitStatus);
+ contribution.setExitStatus(new ExitStatus(exitStatus));
+ return RepeatStatus.FINISHED;
+ }, transactionManager)
+ .build();
+ }
+
+ @Bean
+ @JobScope
+ public Step stepTwo(JobRepository jobRepository, PlatformTransactionManager transactionManager,
+ @Value("#{jobParameters['stepTwo.name']}") String name) {
+ return new StepBuilder(name, jobRepository)
+ .tasklet((StepContribution contribution, ChunkContext chunkContext) -> {
+ logger.info("Hello from stepTwo");
+ return RepeatStatus.FINISHED;
+ }, transactionManager)
+ .build();
+ }
+
+ @Bean
+ public Step stepThree(JobRepository jobRepository, PlatformTransactionManager transactionManager) {
+ return new StepBuilder(SHARED_NAME, jobRepository)
+ .tasklet((StepContribution contribution, ChunkContext chunkContext) -> {
+ logger.info("Hello from stepThree");
+ return RepeatStatus.FINISHED;
+ }, transactionManager)
+ .build();
+ }
+
+ @Bean
+ public Step stepFour(JobRepository jobRepository, PlatformTransactionManager transactionManager) {
+ return new StepBuilder(SHARED_NAME, jobRepository)
+ .tasklet((StepContribution contribution, ChunkContext chunkContext) -> {
+ logger.info("Hello from stepFour");
+ return RepeatStatus.FINISHED;
+ }, transactionManager)
+ .build();
+ }
+
+ @Bean
+ public Job job(JobRepository jobRepository, @Qualifier("conditionalStep") Step conditionalStep,
+ @Qualifier("stepFour") Step step4, @Qualifier("stepTwo") Step step2,
+ @Qualifier("stepThree") Step step3) {
+ JobBuilder jobBuilder = new JobBuilder("flow", jobRepository);
+ return jobBuilder.start(conditionalStep)
+ .on("ODD")
+ .to(step2)
+ .from(conditionalStep)
+ .on("EVEN")
+ .to(step3)
+ .from(step3)
+ .next(step4)
+ .from(step2)
+ .next(step4)
+ .end()
+ .build();
+ }
+
+ @Bean
+ public DataSource dataSource() {
+ return new EmbeddedDatabaseBuilder().addScript("/org/springframework/batch/core/schema-drop-hsqldb.sql")
+ .addScript("/org/springframework/batch/core/schema-hsqldb.sql")
+ .generateUniqueName(true)
+ .build();
+ }
+
+ @Bean
+ public JdbcTransactionManager transactionManager(DataSource dataSource) {
+ return new JdbcTransactionManager(dataSource);
+ }
+
+ }
+
@EnableBatchProcessing
@Configuration
static class JobConfiguration {
diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/launch/support/SimpleJobOperatorTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/launch/support/SimpleJobOperatorTests.java
index d5d3951c2c..619c019d6f 100644
--- a/spring-batch-core/src/test/java/org/springframework/batch/core/launch/support/SimpleJobOperatorTests.java
+++ b/spring-batch-core/src/test/java/org/springframework/batch/core/launch/support/SimpleJobOperatorTests.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2006-2023 the original author or authors.
+ * Copyright 2006-2025 the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -455,6 +455,10 @@ protected void doExecute(JobExecution execution) throws JobExecutionException {
}
+ @Override
+ protected void checkStepNamesUnicity() {
+ }
+
}
}