From 2d5335a6d13f959d7450d92fe80b1712f33ac338 Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@insee.fr>
Date: Wed, 24 Jan 2024 23:46:58 +0100
Subject: [PATCH 1/6] test(check unicity of job name in a flow) failing test

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 .../builder/AlreadyUsedStepNameException.java | 33 +++++++
 .../core/job/builder/FlowJobBuilderTests.java | 86 ++++++++++---------
 2 files changed, 79 insertions(+), 40 deletions(-)
 create mode 100644 spring-batch-core/src/main/java/org/springframework/batch/core/job/builder/AlreadyUsedStepNameException.java

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..6866f17314
--- /dev/null
+++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/builder/AlreadyUsedStepNameException.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2006-2024 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/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..2d479c16d4 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
@@ -15,27 +15,10 @@
  */
 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.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;
@@ -56,16 +39,24 @@
 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;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
 /**
  * @author Dave Syer
  * @author Mahmoud Ben Hassine
- *
+ * @author Fabrice Bibonne
  */
 class FlowJobBuilderTests {
 
@@ -267,26 +258,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 +359,41 @@ void testBuildWithJobScopedStep() throws Exception {
 		assertEquals(ExitStatus.COMPLETED, jobExecution.getExitStatus());
 	}
 
+	//https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
+	@Test
+	void testStepNamesMustBeUniqueWithinFlowDefinition() {
+		Step conditionalStep = new StepSupport("conditionalStep") {
+			@Override
+			public void execute(StepExecution stepExecution) {
+				stepExecution.upgradeStatus(BatchStatus.COMPLETED);
+				stepExecution.setExitStatus(ExitStatus.COMPLETED);
+				String exitStatus = (System.currentTimeMillis() % 2 == 0) ? "EVEN" : "ODD";
+				stepExecution.setExitStatus(new ExitStatus(exitStatus));
+				jobRepository.update(stepExecution);
+			}
+		};
+
+		StepSupport misnamedStep = new StepSupport("step3") {
+			@Override
+			public void execute(StepExecution stepExecution)
+					throws UnexpectedJobExecutionException {
+
+				stepExecution.upgradeStatus(BatchStatus.COMPLETED);
+				stepExecution.setExitStatus(ExitStatus.COMPLETED);
+				jobRepository.update(stepExecution);
+			}
+		};
+
+		JobBuilder jobBuilder = new JobBuilder("flow", jobRepository);
+		FlowBuilder<FlowJobBuilder> flowBuilder = jobBuilder.start(conditionalStep)
+				.on("ODD").to(step2)
+				.from(conditionalStep).on("EVEN").to(step3)
+				.from(step3);
+		assertThrows(AlreadyUsedStepNameException.class, () -> flowBuilder.next(misnamedStep));
+		flowBuilder.end().build();
+	}
+
+
 	@EnableBatchProcessing
 	@Configuration
 	static class JobConfiguration {

From fd184c32ab61e82531f7b2b2e14c496abefb2aaa Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@gmail.com>
Date: Mon, 3 Feb 2025 05:50:10 +0100
Subject: [PATCH 2/6] test(check unicity of job name in a flow) step name as a
 job parameter

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 .../builder/AlreadyUsedStepNameException.java |   9 +-
 .../core/job/builder/FlowJobBuilderTests.java | 114 ++++++++++++++----
 2 files changed, 92 insertions(+), 31 deletions(-)

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
index 6866f17314..fd88081717 100644
--- 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
@@ -22,12 +22,11 @@
  * relies on the step name
  *
  * @author Fabrice Bibonne
- *
  */
-public class AlreadyUsedStepNameException extends RuntimeException{
+public class AlreadyUsedStepNameException extends RuntimeException {
 
-    public AlreadyUsedStepNameException(String name){
-        super("the name "+name+" is already used");
-    }
+	public AlreadyUsedStepNameException(String name) {
+		super("the name " + name + " is already used");
+	}
 
 }
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 2d479c16d4..529964bc55 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
@@ -15,6 +15,8 @@
  */
 package org.springframework.batch.core.job.builder;
 
+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;
@@ -28,11 +30,13 @@
 import org.springframework.batch.core.launch.JobLauncher;
 import org.springframework.batch.core.repository.JobRepository;
 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;
@@ -362,35 +366,93 @@ void testBuildWithJobScopedStep() throws Exception {
 	//https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
 	@Test
 	void testStepNamesMustBeUniqueWithinFlowDefinition() {
-		Step conditionalStep = new StepSupport("conditionalStep") {
-			@Override
-			public void execute(StepExecution stepExecution) {
-				stepExecution.upgradeStatus(BatchStatus.COMPLETED);
-				stepExecution.setExitStatus(ExitStatus.COMPLETED);
-				String exitStatus = (System.currentTimeMillis() % 2 == 0) ? "EVEN" : "ODD";
-				stepExecution.setExitStatus(new ExitStatus(exitStatus));
-				jobRepository.update(stepExecution);
-			}
-		};
+		ApplicationContext context = new AnnotationConfigApplicationContext(JobConfigurationForStepNameUnique.class);
+		JobLauncher jobLauncher = context.getBean(JobLauncher.class);
+		Job job = context.getBean(Job.class);
+		assertThrows(AlreadyUsedStepNameException.class, ()->jobLauncher.run(job, new JobParametersBuilder().addLong("random", 2L).addString("stepTwo.name", JobConfigurationForStepNameUnique.SHARED_NAME).toJobParameters()));
+		assertThrows(AlreadyUsedStepNameException.class, ()->jobLauncher.run(job, new JobParametersBuilder().addLong("random", 1L).addString("stepTwo.name",JobConfigurationForStepNameUnique.SHARED_NAME).toJobParameters()));
+	}
 
-		StepSupport misnamedStep = new StepSupport("step3") {
-			@Override
-			public void execute(StepExecution stepExecution)
-					throws UnexpectedJobExecutionException {
+	@EnableBatchProcessing
+	@Configuration
+	static class JobConfigurationForStepNameUnique{
 
-				stepExecution.upgradeStatus(BatchStatus.COMPLETED);
-				stepExecution.setExitStatus(ExitStatus.COMPLETED);
-				jobRepository.update(stepExecution);
-			}
-		};
+		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();
+		}
 
-		JobBuilder jobBuilder = new JobBuilder("flow", jobRepository);
-		FlowBuilder<FlowJobBuilder> flowBuilder = jobBuilder.start(conditionalStep)
-				.on("ODD").to(step2)
-				.from(conditionalStep).on("EVEN").to(step3)
-				.from(step3);
-		assertThrows(AlreadyUsedStepNameException.class, () -> flowBuilder.next(misnamedStep));
-		flowBuilder.end().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);
+		}
 	}
 
 

From 9c6c392f99c10754a1d4494612f68aa0f42c6345 Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@insee.fr>
Date: Thu, 1 Feb 2024 18:20:19 +0100
Subject: [PATCH 3/6] fix(log for tests) print log in console for tests

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 pom.xml | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/pom.xml b/pom.xml
index eb01fbdcc5..348c8d6771 100644
--- a/pom.xml
+++ b/pom.xml
@@ -152,6 +152,26 @@
 		<spring-javaformat-maven-plugin.version>0.0.39</spring-javaformat-maven-plugin.version>
 	</properties>
 
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>jcl-over-slf4j</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-api</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-simple</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
+
 	<build>
 		<plugins>
 			<plugin>

From 6a5a14b314ded3e9eaf1c5c7d1ab5785bde92d9f Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@insee.fr>
Date: Mon, 5 Feb 2024 22:05:59 +0100
Subject: [PATCH 4/6] feat(step name unicity) #3757 the names of different
 steps in a job must be different

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 .../batch/core/job/AbstractJob.java           | 30 +++++++++-
 .../batch/core/job/SimpleJob.java             | 11 +++-
 .../batch/core/job/flow/FlowJob.java          | 55 +++++++++++--------
 .../core/job/ExtendedAbstractJobTests.java    |  7 +++
 .../core/job/builder/FlowJobBuilderTests.java | 16 ++++--
 .../support/SimpleJobOperatorTests.java       |  5 ++
 6 files changed, 91 insertions(+), 33 deletions(-)

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..3b98aafb2b 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
@@ -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,9 +369,23 @@ public final void execute(JobExecution execution) {
 			finally {
 				JobSynchronizationManager.release();
 			}
-
 		}
+	}
+
+	protected abstract void checkStepNamesUnicity() throws AlreadyUsedStepNameException ;
 
+	private Optional<String> findFirstDoubleElementInList(List<String> strings) {
+		if (strings==null){
+			return Optional.empty();
+		}
+		Set<String> alreadyChecked=new HashSet<>();
+		for (String value:strings){
+			if (alreadyChecked.contains(value)){
+				return Optional.of(value);
+			}
+			alreadyChecked.add(value);
+		}
+		return Optional.empty();
 	}
 
 	private void stopObservation(JobExecution execution, Observation observation) {
@@ -430,6 +445,15 @@ else if (ex instanceof NoSuchJobException || ex.getCause() instanceof NoSuchJobE
 		return exitStatus;
 	}
 
+	protected static void addToMapCheckingUnicity(Map<String, Step> 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..94aa2911de 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
@@ -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;
@@ -27,6 +25,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.repository.JobRestartException;
 import org.springframework.batch.core.step.StepLocator;
 
@@ -145,4 +144,10 @@ protected void doExecute(JobExecution execution)
 		}
 	}
 
+	@Override
+	protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
+		Map<String, Step> map = new HashMap<>();
+		steps.forEach(step->{addToMapCheckingUnicity(map, step, step.getName());});
+	}
+
 }
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..37405db252 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
@@ -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,40 +75,47 @@ public void setFlow(Flow flow) {
 	 */
 	@Override
 	public Step getStep(String stepName) {
-		if (!initialized) {
-			init();
-		}
+		init();
 		return stepMap.get(stepName);
 	}
 
+
 	/**
 	 * Initialize the step names
 	 */
 	private void init() {
-		findSteps(flow, stepMap);
-		initialized = true;
+		if (!initialized) {
+			findStepsThrowingIfNameNotUnique(flow, stepMap);
+			initialized = true;
+		}
 	}
 
-	private void findSteps(Flow flow, Map<String, Step> map) {
+	private void findStepsThrowingIfNameNotUnique(Flow flow, Map<String, Step> map) {
 
 		for (State state : flow.getStates()) {
 			if (state instanceof StepLocator locator) {
 				for (String name : locator.getStepNames()) {
-					map.put(name, locator.getStep(name));
+					addToMapCheckingUnicity(map, 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 wich implements StepHolder is StepState which implements also StepLocator
+			/*
+			Tests Coverage
+			Hits : 30
+				state instanceof StepHolder
+					true hits: 0
+					false hits : 30
+			*/
+			else if (state instanceof StepHolder stepHolder) {
+				Step step = stepHolder.getStep();
+				addToMapCheckingUnicity(map, 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, map);
 				}
 			}
 		}
-
 	}
 
 	/**
@@ -115,9 +123,7 @@ else if (state instanceof FlowHolder) {
 	 */
 	@Override
 	public Collection<String> getStepNames() {
-		if (!initialized) {
-			init();
-		}
+		init();
 		return stepMap.keySet();
 	}
 
@@ -139,4 +145,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..cf04807204 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
@@ -25,6 +25,7 @@
 import org.springframework.batch.core.JobParametersInvalidException;
 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.repository.JobRepository;
 import org.springframework.batch.core.repository.support.JobRepositoryFactoryBean;
 import org.springframework.batch.core.step.StepSupport;
@@ -36,6 +37,7 @@
 import java.time.LocalDateTime;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -215,6 +217,11 @@ public StubJob() {
 		protected void doExecute(JobExecution execution) throws JobExecutionException {
 		}
 
+		@Override
+		protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
+
+		}
+
 		@Override
 		public Step getStep(String stepName) {
 			return null;
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 529964bc55..d06b7c3a2b 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
@@ -28,7 +28,10 @@
 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;
@@ -54,8 +57,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.*;
 
 /**
  * @author Dave Syer
@@ -365,12 +367,16 @@ void testBuildWithJobScopedStep() throws Exception {
 
 	//https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
 	@Test
-	void testStepNamesMustBeUniqueWithinFlowDefinition() {
+	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);
-		assertThrows(AlreadyUsedStepNameException.class, ()->jobLauncher.run(job, new JobParametersBuilder().addLong("random", 2L).addString("stepTwo.name", JobConfigurationForStepNameUnique.SHARED_NAME).toJobParameters()));
-		assertThrows(AlreadyUsedStepNameException.class, ()->jobLauncher.run(job, new JobParametersBuilder().addLong("random", 1L).addString("stepTwo.name",JobConfigurationForStepNameUnique.SHARED_NAME).toJobParameters()));
+		JobExecution jobExecution=jobLauncher.run(job, new JobParametersBuilder().addLong("random", 2L).addString("stepTwo.name", JobConfigurationForStepNameUnique.SHARED_NAME).toJobParameters());
+		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());
+		assertTrue(jobExecution.getAllFailureExceptions().stream().map(Object::getClass).anyMatch(AlreadyUsedStepNameException.class::equals));
+		assertEquals(ExitStatus.FAILED.getExitCode(), jobExecution.getExitStatus().getExitCode());
 	}
 
 	@EnableBatchProcessing
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..daf961cbc1 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
@@ -43,6 +43,7 @@
 import org.springframework.batch.core.explore.JobExplorer;
 import org.springframework.batch.core.job.AbstractJob;
 import org.springframework.batch.core.job.JobSupport;
+import org.springframework.batch.core.job.builder.AlreadyUsedStepNameException;
 import org.springframework.batch.core.launch.JobInstanceAlreadyExistsException;
 import org.springframework.batch.core.launch.NoSuchJobException;
 import org.springframework.batch.core.launch.NoSuchJobExecutionException;
@@ -455,6 +456,10 @@ protected void doExecute(JobExecution execution) throws JobExecutionException {
 
 		}
 
+		@Override
+		protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
+		}
+
 	}
 
 }

From 1afb9ab3c4baa49217af36db5e820e6ffe64f781 Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@insee.fr>
Date: Tue, 6 Feb 2024 06:30:33 +0100
Subject: [PATCH 5/6] fix(step name unicity WITHIN A FLOW)

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 .../batch/core/job/AbstractJob.java           | 14 ------------
 .../batch/core/job/SimpleJob.java             |  6 ++---
 .../batch/core/job/flow/FlowJob.java          | 12 +++++-----
 .../core/job/ExtendedAbstractJobTests.java    | 22 ++++---------------
 .../batch/core/job/SimpleJobTests.java        | 19 ++++++++++++++++
 .../core/job/builder/FlowJobBuilderTests.java |  8 +++----
 .../support/SimpleJobOperatorTests.java       |  3 +--
 7 files changed, 36 insertions(+), 48 deletions(-)

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 3b98aafb2b..29b01314f6 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
@@ -374,20 +374,6 @@ public final void execute(JobExecution execution) {
 
 	protected abstract void checkStepNamesUnicity() throws AlreadyUsedStepNameException ;
 
-	private Optional<String> findFirstDoubleElementInList(List<String> strings) {
-		if (strings==null){
-			return Optional.empty();
-		}
-		Set<String> alreadyChecked=new HashSet<>();
-		for (String value:strings){
-			if (alreadyChecked.contains(value)){
-				return Optional.of(value);
-			}
-			alreadyChecked.add(value);
-		}
-		return Optional.empty();
-	}
-
 	private void stopObservation(JobExecution execution, Observation observation) {
 		List<Throwable> throwables = execution.getFailureExceptions();
 		if (!throwables.isEmpty()) {
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 94aa2911de..7b90d3931f 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
@@ -25,7 +25,6 @@
 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.repository.JobRestartException;
 import org.springframework.batch.core.step.StepLocator;
 
@@ -145,9 +144,8 @@ protected void doExecute(JobExecution execution)
 	}
 
 	@Override
-	protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
-		Map<String, Step> map = new HashMap<>();
-		steps.forEach(step->{addToMapCheckingUnicity(map, step, step.getName());});
+	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/flow/FlowJob.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/flow/FlowJob.java
index 37405db252..6ce84fc8a2 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
@@ -85,20 +85,20 @@ public Step getStep(String stepName) {
 	 */
 	private void init() {
 		if (!initialized) {
-			findStepsThrowingIfNameNotUnique(flow, stepMap);
+			findStepsThrowingIfNameNotUnique(flow);
 			initialized = true;
 		}
 	}
 
-	private void findStepsThrowingIfNameNotUnique(Flow flow, Map<String, Step> map) {
+	private void findStepsThrowingIfNameNotUnique(Flow flow) {
 
 		for (State state : flow.getStates()) {
 			if (state instanceof StepLocator locator) {
 				for (String name : locator.getStepNames()) {
-					addToMapCheckingUnicity(map, locator.getStep(name), name);
+					addToMapCheckingUnicity(this.stepMap, locator.getStep(name), name);
 				}
 			}
-			//TODO remove this else bock ? not executed during tests : the only State wich implements StepHolder is StepState which implements also StepLocator
+			//TODO remove this else bock ? not executed during tests : the only State which implements StepHolder is StepState which already implements StepLocator
 			/*
 			Tests Coverage
 			Hits : 30
@@ -108,11 +108,11 @@ private void findStepsThrowingIfNameNotUnique(Flow flow, Map<String, Step> map)
 			*/
 			else if (state instanceof StepHolder stepHolder) {
 				Step step = stepHolder.getStep();
-				addToMapCheckingUnicity(map, step, step.getName());
+				addToMapCheckingUnicity(this.stepMap, step, step.getName());
 			}
 			else if (state instanceof FlowHolder flowHolder) {
 				for (Flow subflow : flowHolder.getFlows()) {
-					findStepsThrowingIfNameNotUnique(subflow, map);
+					findStepsThrowingIfNameNotUnique(subflow);
 				}
 			}
 		}
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 cf04807204..96ca98c68d 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
@@ -17,33 +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.job.builder.AlreadyUsedStepNameException;
+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 java.util.List;
 
-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
@@ -218,8 +205,7 @@ protected void doExecute(JobExecution execution) throws JobExecutionException {
 		}
 
 		@Override
-		protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
-
+		protected void checkStepNamesUnicity(){
 		}
 
 		@Override
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..7d511dcb4d 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
@@ -513,6 +513,25 @@ void testGetMultipleJobParameters() throws Exception {
 
 	}
 
+	@Test
+	public void testMultipleStepsWithSameName(){
+		job.setName("MultipleStepsWithSameName");
+		String sharedName="stepName";
+		final List<String> 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 d06b7c3a2b..921c0ad1d1 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-2023 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.
@@ -57,7 +57,7 @@
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * @author Dave Syer
@@ -372,10 +372,10 @@ void testStepNamesMustBeUniqueWithinFlowDefinition() throws JobInstanceAlreadyCo
 		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());
-		assertTrue(jobExecution.getAllFailureExceptions().stream().map(Object::getClass).anyMatch(AlreadyUsedStepNameException.class::equals));
+		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());
-		assertTrue(jobExecution.getAllFailureExceptions().stream().map(Object::getClass).anyMatch(AlreadyUsedStepNameException.class::equals));
+		Assertions.assertTrue(jobExecution.getAllFailureExceptions().stream().map(Object::getClass).anyMatch(AlreadyUsedStepNameException.class::equals));
 		assertEquals(ExitStatus.FAILED.getExitCode(), jobExecution.getExitStatus().getExitCode());
 	}
 
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 daf961cbc1..e61a8fb5a1 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
@@ -43,7 +43,6 @@
 import org.springframework.batch.core.explore.JobExplorer;
 import org.springframework.batch.core.job.AbstractJob;
 import org.springframework.batch.core.job.JobSupport;
-import org.springframework.batch.core.job.builder.AlreadyUsedStepNameException;
 import org.springframework.batch.core.launch.JobInstanceAlreadyExistsException;
 import org.springframework.batch.core.launch.NoSuchJobException;
 import org.springframework.batch.core.launch.NoSuchJobExecutionException;
@@ -457,7 +456,7 @@ protected void doExecute(JobExecution execution) throws JobExecutionException {
 		}
 
 		@Override
-		protected void checkStepNamesUnicity() throws AlreadyUsedStepNameException {
+		protected void checkStepNamesUnicity() {
 		}
 
 	}

From 03c822357482d2a70b0ecbacb3b40a94853d2e3f Mon Sep 17 00:00:00 2001
From: Fabrice Bibonne <fabrice.bibonne@insee.fr>
Date: Tue, 6 Feb 2024 06:44:37 +0100
Subject: [PATCH 6/6] fix(java format)   Also change copyright date to 2025

Signed-off-by: Fabrice Bibonne <fabrice.bibonne@gmail.com>
---
 .../batch/core/job/AbstractJob.java           |  11 +-
 .../batch/core/job/SimpleJob.java             |   4 +-
 .../builder/AlreadyUsedStepNameException.java |   8 +-
 .../batch/core/job/flow/FlowJob.java          |  15 +--
 .../core/job/ExtendedAbstractJobTests.java    |   4 +-
 .../batch/core/job/SimpleJobTests.java        |  20 ++--
 .../core/job/builder/FlowJobBuilderTests.java | 110 ++++++++++--------
 .../support/SimpleJobOperatorTests.java       |   2 +-
 8 files changed, 94 insertions(+), 80 deletions(-)

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 29b01314f6..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.
@@ -372,7 +372,7 @@ public final void execute(JobExecution execution) {
 		}
 	}
 
-	protected abstract void checkStepNamesUnicity() throws AlreadyUsedStepNameException ;
+	protected abstract void checkStepNamesUnicity() throws AlreadyUsedStepNameException;
 
 	private void stopObservation(JobExecution execution, Observation observation) {
 		List<Throwable> throwables = execution.getFailureExceptions();
@@ -431,9 +431,10 @@ else if (ex instanceof NoSuchJobException || ex.getCause() instanceof NoSuchJobE
 		return exitStatus;
 	}
 
-	protected static void addToMapCheckingUnicity(Map<String, Step> map, Step step, String name) throws AlreadyUsedStepNameException {
-		map.merge(name, step, (old, value)->{
-			if (!old.equals(value)){
+	protected static void addToMapCheckingUnicity(Map<String, Step> map, Step step, String name)
+			throws AlreadyUsedStepNameException {
+		map.merge(name, step, (old, value) -> {
+			if (!old.equals(value)) {
 				throw new AlreadyUsedStepNameException(name);
 			}
 			return old;
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 7b90d3931f..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.
@@ -145,7 +145,7 @@ protected void doExecute(JobExecution execution)
 
 	@Override
 	protected void checkStepNamesUnicity() {
-		//noop : steps of SimpleJob can share the same name
+		// 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
index fd88081717..4547447e46 100644
--- 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
@@ -1,5 +1,5 @@
 /*
- * Copyright 2006-2024 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,9 +17,9 @@
 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
+ * 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
  */
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 6ce84fc8a2..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.
@@ -79,7 +79,6 @@ public Step getStep(String stepName) {
 		return stepMap.get(stepName);
 	}
 
-
 	/**
 	 * Initialize the step names
 	 */
@@ -98,14 +97,10 @@ private void findStepsThrowingIfNameNotUnique(Flow flow) {
 					addToMapCheckingUnicity(this.stepMap, locator.getStep(name), name);
 				}
 			}
-			//TODO remove this else bock ? not executed during tests : the only State which implements StepHolder is StepState which already implements StepLocator
-			/*
-			Tests Coverage
-			Hits : 30
-				state instanceof StepHolder
-					true hits: 0
-					false hits : 30
-			*/
+			// 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());
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 96ca98c68d..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.
@@ -205,7 +205,7 @@ protected void doExecute(JobExecution execution) throws JobExecutionException {
 		}
 
 		@Override
-		protected void checkStepNamesUnicity(){
+		protected void checkStepNamesUnicity() {
 		}
 
 		@Override
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 7d511dcb4d..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.
@@ -514,18 +514,18 @@ void testGetMultipleJobParameters() throws Exception {
 	}
 
 	@Test
-	public void testMultipleStepsWithSameName(){
+	public void testMultipleStepsWithSameName() {
 		job.setName("MultipleStepsWithSameName");
-		String sharedName="stepName";
-		final List<String> executionsCallbacks=new ArrayList<>();
-		StubStep sharedNameStep1=new StubStep(sharedName, jobRepository);
-		sharedNameStep1.setCallback(()->executionsCallbacks.add("step1"));
+		String sharedName = "stepName";
+		final List<String> 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"));
+		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"));
+		StubStep sharedNameStep3 = new StubStep(sharedName, jobRepository);
+		sharedNameStep3.setCallback(() -> executionsCallbacks.add("step3"));
 		job.addStep(sharedNameStep3);
 		job.execute(jobExecution);
 		assertEquals(List.of("step1", "step2", "step3"), executionsCallbacks);
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 921c0ad1d1..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-2023 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.
@@ -365,102 +365,120 @@ void testBuildWithJobScopedStep() throws Exception {
 		assertEquals(ExitStatus.COMPLETED, jobExecution.getExitStatus());
 	}
 
-	//https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
+	// https://github.com/spring-projects/spring-batch/issues/3757#issuecomment-1821593539
 	@Test
-	void testStepNamesMustBeUniqueWithinFlowDefinition() throws JobInstanceAlreadyCompleteException, JobExecutionAlreadyRunningException, JobParametersInvalidException, JobRestartException {
+	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));
+		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));
+		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{
+	static class JobConfigurationForStepNameUnique {
 
-		private static final String SHARED_NAME ="sharedName";
+		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();
+				@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) {
+				@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();
+				.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();
+				.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();
+				.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) {
+				@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();
+				.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();
+				.addScript("/org/springframework/batch/core/schema-hsqldb.sql")
+				.generateUniqueName(true)
+				.build();
 		}
 
 		@Bean
 		public JdbcTransactionManager transactionManager(DataSource dataSource) {
 			return new JdbcTransactionManager(dataSource);
 		}
-	}
 
+	}
 
 	@EnableBatchProcessing
 	@Configuration
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 e61a8fb5a1..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.