diff --git a/pom.xml b/pom.xml index d32aba8d05..65a30af2b0 100644 --- a/pom.xml +++ b/pom.xml @@ -55,6 +55,7 @@ 1.3.1 6.0.0-SNAPSHOT 2.0.0-SNAPSHOT + 1.0.0-SNAPSHOT 2.13.1 diff --git a/spring-batch-core/pom.xml b/spring-batch-core/pom.xml index 3d14bf9f99..49a952bdaf 100644 --- a/spring-batch-core/pom.xml +++ b/spring-batch-core/pom.xml @@ -256,6 +256,12 @@ ${jakarta.inject-api.version} test + + io.micrometer + micrometer-test + ${micrometer.version} + test + 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 5b3ac1502a..4a591a9048 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 @@ -18,10 +18,12 @@ import java.util.Collection; import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; -import io.micrometer.api.instrument.LongTaskTimer; -import io.micrometer.api.instrument.Tag; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.LongTaskTimer; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.observation.Observation; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.springframework.batch.core.BatchStatus; @@ -62,7 +64,7 @@ * @author Mahmoud Ben Hassine */ public abstract class AbstractJob implements Job, StepLocator, BeanNameAware, -InitializingBean { +InitializingBean, Observation.TagsProviderAware { protected static final Log logger = LogFactory.getLog(AbstractJob.class); @@ -80,6 +82,8 @@ public abstract class AbstractJob implements Job, StepLocator, BeanNameAware, private StepHandler stepHandler; + private BatchJobTagsProvider tagsProvider = new DefaultBatchJobTagsProvider(); + /** * Default constructor. */ @@ -304,8 +308,11 @@ public final void execute(JobExecution execution) { LongTaskTimer longTaskTimer = BatchMetrics.createLongTaskTimer("job.active", "Active jobs", Tag.of("name", execution.getJobInstance().getJobName())); LongTaskTimer.Sample longTaskTimerSample = longTaskTimer.start(); - Timer.Sample timerSample = BatchMetrics.createTimerSample(); - try { + Observation observation = BatchMetrics.createObservation(BatchJobObservation.BATCH_JOB_OBSERVATION.getName(), new BatchJobContext(execution)) + .contextualName(execution.getJobInstance().getJobName()) + .tagsProvider(this.tagsProvider) + .start(); + try (Observation.Scope scope = observation.openScope()) { jobParametersValidator.validate(execution.getJobParameters()); @@ -361,11 +368,7 @@ public final void execute(JobExecution execution) { ExitStatus.NOOP.addExitDescription("All steps already completed or no steps configured for this job."); execution.setExitStatus(exitStatus.and(newExitStatus)); } - - timerSample.stop(BatchMetrics.createTimer("job", "Job duration", - Tag.of("name", execution.getJobInstance().getJobName()), - Tag.of("status", execution.getExitStatus().getExitCode()) - )); + stopObservation(execution, observation); longTaskTimerSample.stop(); execution.setEndTime(new Date()); @@ -384,6 +387,19 @@ public final void execute(JobExecution execution) { } + private void stopObservation(JobExecution execution, Observation observation) { + List throwables = execution.getFailureExceptions(); + if (!throwables.isEmpty()) { + observation.error(mergedThrowables(throwables)); + } + observation.stop(); + } + + private IllegalStateException mergedThrowables(List throwables) { + return new IllegalStateException( + throwables.stream().map(Throwable::toString).collect(Collectors.joining("\n"))); + } + /** * Convenience method for subclasses to delegate the handling of a specific * step in the context of the current {@link JobExecution}. Clients of this @@ -443,6 +459,11 @@ private void updateStatus(JobExecution jobExecution, BatchStatus status) { jobRepository.update(jobExecution); } + @Override + public void setTagsProvider(BatchJobTagsProvider tagsProvider) { + this.tagsProvider = tagsProvider; + } + @Override public String toString() { return ClassUtils.getShortName(getClass()) + ": [name=" + name + "]"; diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobContext.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobContext.java new file mode 100644 index 0000000000..c3866c3347 --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobContext.java @@ -0,0 +1,35 @@ +/* + * Copyright 2013-2021 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; + +import io.micrometer.core.instrument.observation.Observation; + +import org.springframework.batch.core.JobExecution; + +public class BatchJobContext extends Observation.Context { + + private final JobExecution jobExecution; + + public BatchJobContext(JobExecution jobExecution) { + this.jobExecution = jobExecution; + } + + public JobExecution getJobExecution() { + return jobExecution; + } + +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobObservation.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobObservation.java new file mode 100644 index 0000000000..c3fa1f49ab --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobObservation.java @@ -0,0 +1,101 @@ +/* + * Copyright 2013-2021 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; + +import io.micrometer.core.instrument.docs.DocumentedObservation; +import io.micrometer.core.instrument.docs.TagKey; + +enum BatchJobObservation implements DocumentedObservation { + + /** + * Observation created around a Job execution. + */ + BATCH_JOB_OBSERVATION { + @Override + public String getName() { + return "spring.batch.job"; + } + + @Override + public String getContextualName() { + return "%s"; + } + + @Override + public TagKey[] getLowCardinalityTagKeys() { + return JobLowCardinalityTags.values(); + } + + @Override + public TagKey[] getHighCardinalityTagKeys() { + return JobHighCardinalityTags.values(); + } + + @Override + public String getPrefix() { + return "spring.batch"; + } + }; + + enum JobLowCardinalityTags implements TagKey { + + /** + * Name of the Spring Batch job. + */ + JOB_NAME { + @Override + public String getKey() { + return "spring.batch.job.name"; + } + }, + + /** + * Job status. + */ + JOB_STATUS { + @Override + public String getKey() { + return "spring.batch.job.status"; + } + } + + } + + enum JobHighCardinalityTags implements TagKey { + + /** + * ID of the Spring Batch job instance. + */ + JOB_INSTANCE_ID { + @Override + public String getKey() { + return "spring.batch.job.instanceId"; + } + }, + + /** + * ID of the Spring Batch execution. + */ + JOB_EXECUTION_ID { + @Override + public String getKey() { + return "spring.batch.job.executionId"; + } + } + + } +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobTagsProvider.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobTagsProvider.java new file mode 100644 index 0000000000..41bcf7ea1b --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/BatchJobTagsProvider.java @@ -0,0 +1,40 @@ +/* + * Copyright 2006-2009 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; + +import io.micrometer.core.instrument.observation.Observation; + +import org.springframework.batch.core.Job; +import org.springframework.batch.core.JobExecution; +import org.springframework.batch.core.JobInterruptedException; +import org.springframework.batch.core.StartLimitExceededException; +import org.springframework.batch.core.Step; +import org.springframework.batch.core.StepExecution; +import org.springframework.batch.core.repository.JobRestartException; + +/** + * {@link Observation.TagsProvider} for {@link BatchJobContext}. + * + * @author Marcin Grzejszczak + */ +public interface BatchJobTagsProvider extends Observation.TagsProvider { + + @Override + default boolean supportsContext(Observation.Context context) { + return context instanceof BatchJobContext; + } +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/job/DefaultBatchJobTagsProvider.java b/spring-batch-core/src/main/java/org/springframework/batch/core/job/DefaultBatchJobTagsProvider.java new file mode 100644 index 0000000000..7c6497459d --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/job/DefaultBatchJobTagsProvider.java @@ -0,0 +1,42 @@ +/* + * Copyright 2011-2018 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; + +import io.micrometer.core.instrument.Tags; + +import org.springframework.batch.core.JobExecution; + +/** + * Default {@link BatchJobTagsProvider} implementation. + * + * @author Marcin Grzejszczak + */ +public class DefaultBatchJobTagsProvider implements BatchJobTagsProvider { + @Override + public Tags getLowCardinalityTags(BatchJobContext context) { + JobExecution execution = context.getJobExecution(); + return Tags.of(BatchJobObservation.JobLowCardinalityTags.JOB_NAME.of(execution.getJobInstance().getJobName()), + BatchJobObservation.JobLowCardinalityTags.JOB_STATUS.of(execution.getExitStatus().getExitCode())); + } + + @Override + public Tags getHighCardinalityTags(BatchJobContext context) { + JobExecution execution = context.getJobExecution(); + return Tags.of(BatchJobObservation.JobHighCardinalityTags.JOB_INSTANCE_ID.of(String.valueOf(execution.getJobInstance().getInstanceId())), + BatchJobObservation.JobHighCardinalityTags.JOB_EXECUTION_ID.of(String.valueOf(execution.getId()))); + } + +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/metrics/BatchMetrics.java b/spring-batch-core/src/main/java/org/springframework/batch/core/metrics/BatchMetrics.java index 8ac8c28bc7..61761186b8 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/metrics/BatchMetrics.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/metrics/BatchMetrics.java @@ -20,10 +20,12 @@ import java.util.Date; import java.util.concurrent.TimeUnit; -import io.micrometer.api.instrument.LongTaskTimer; -import io.micrometer.api.instrument.Metrics; -import io.micrometer.api.instrument.Tag; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.LongTaskTimer; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Timer; +import io.micrometer.core.instrument.observation.Observation; +import io.micrometer.core.instrument.observation.TimerObservationHandler; import org.springframework.lang.Nullable; @@ -66,6 +68,19 @@ public static Timer createTimer(String name, String description, Tag... tags) { .register(Metrics.globalRegistry); } + /** + * Create a new {@link Observation}. It's not started, you must + * explicitly call {@link Observation#start()} to start it. + * + * Remember to register the {@link TimerObservationHandler} + * via the {@code Metrics.globalRegistry.withTimerObservationHandler()} + * in the user code. Otherwise you won't observe any metrics. + * @return a new observation instance + */ + public static Observation createObservation(String name, Observation.Context context) { + return Observation.createNotStarted(name, context, Metrics.globalRegistry); + } + /** * Create a new {@link Timer.Sample}. * @return a new timer sample instance diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/AbstractStep.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/AbstractStep.java index 91b5d519d2..1aaaff4913 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/step/AbstractStep.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/AbstractStep.java @@ -17,9 +17,10 @@ import java.time.Duration; import java.util.Date; +import java.util.List; +import java.util.stream.Collectors; -import io.micrometer.api.instrument.Tag; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.observation.Observation; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.springframework.batch.core.BatchStatus; @@ -54,7 +55,7 @@ * @author Chris Schaefer * @author Mahmoud Ben Hassine */ -public abstract class AbstractStep implements Step, InitializingBean, BeanNameAware { +public abstract class AbstractStep implements Step, InitializingBean, BeanNameAware, Observation.TagsProviderAware { private static final Log logger = LogFactory.getLog(AbstractStep.class); @@ -68,6 +69,8 @@ public abstract class AbstractStep implements Step, InitializingBean, BeanNameAw private JobRepository jobRepository; + private BatchStepTagsProvider tagsProvider = new DefaultBatchStepTagsProvider(); + /** * Default constructor. */ @@ -192,7 +195,10 @@ public final void execute(StepExecution stepExecution) throws JobInterruptedExce } stepExecution.setStartTime(new Date()); stepExecution.setStatus(BatchStatus.STARTED); - Timer.Sample sample = BatchMetrics.createTimerSample(); + Observation observation = BatchMetrics.createObservation(BatchStepObservation.BATCH_STEP_OBSERVATION.getName(), new BatchStepContext(stepExecution)) + .contextualName(stepExecution.getStepName()) + .tagsProvider(this.tagsProvider) + .start(); getJobRepository().update(stepExecution); // Start with a default value that will be trumped by anything @@ -200,7 +206,7 @@ public final void execute(StepExecution stepExecution) throws JobInterruptedExce doExecutionRegistration(stepExecution); - try { + try (Observation.Scope scope = observation.openScope()) { getCompositeListener().beforeStep(stepExecution); open(stepExecution.getExecutionContext()); @@ -260,12 +266,7 @@ public final void execute(StepExecution stepExecution) throws JobInterruptedExce logger.error(String.format("Encountered an error saving batch meta data for step %s in job %s. " + "This job is now in an unknown state and should not be restarted.", name, stepExecution.getJobExecution().getJobInstance().getJobName()), e); } - - sample.stop(BatchMetrics.createTimer("step", "Step duration", - Tag.of("job.name", stepExecution.getJobExecution().getJobInstance().getJobName()), - Tag.of("name", stepExecution.getStepName()), - Tag.of("status", stepExecution.getExitStatus().getExitCode()) - )); + stopObservation(stepExecution, observation); stepExecution.setEndTime(new Date()); stepExecution.setExitStatus(exitStatus); Duration stepExecutionDuration = BatchMetrics.calculateDuration(stepExecution.getStartTime(), stepExecution.getEndTime()); @@ -299,6 +300,19 @@ public final void execute(StepExecution stepExecution) throws JobInterruptedExce } } + private void stopObservation(StepExecution stepExecution, Observation observation) { + List throwables = stepExecution.getFailureExceptions(); + if (!throwables.isEmpty()) { + observation.error(mergedThrowables(throwables)); + } + observation.stop(); + } + + private IllegalStateException mergedThrowables(List throwables) { + return new IllegalStateException( + throwables.stream().map(Throwable::toString).collect(Collectors.joining("\n"))); + } + /** * Releases the most recent {@link StepExecution} */ @@ -394,4 +408,8 @@ else if (ex instanceof NoSuchJobException || ex.getCause() instanceof NoSuchJobE return exitStatus; } + @Override + public void setTagsProvider(BatchStepTagsProvider tagsProvider) { + this.tagsProvider = tagsProvider; + } } diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepContext.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepContext.java new file mode 100644 index 0000000000..7865ea43d5 --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepContext.java @@ -0,0 +1,35 @@ +/* + * Copyright 2013-2021 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.step; + +import io.micrometer.core.instrument.observation.Observation; + +import org.springframework.batch.core.StepExecution; + +public class BatchStepContext extends Observation.Context { + + private final StepExecution stepExecution; + + public BatchStepContext(StepExecution stepExecution) { + this.stepExecution = stepExecution; + } + + public StepExecution getStepExecution() { + return stepExecution; + } + +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepObservation.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepObservation.java new file mode 100644 index 0000000000..35a7312d0b --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepObservation.java @@ -0,0 +1,112 @@ +/* + * Copyright 2013-2021 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.step; + +import io.micrometer.core.instrument.docs.DocumentedObservation; +import io.micrometer.core.instrument.docs.TagKey; + +enum BatchStepObservation implements DocumentedObservation { + + /** + * Observation created around a Job execution. + */ + BATCH_STEP_OBSERVATION { + @Override + public String getName() { + return "spring.batch.step"; + } + + @Override + public String getContextualName() { + return "%s"; + } + + @Override + public TagKey[] getLowCardinalityTagKeys() { + return StepLowCardinalityTags.values(); + } + + @Override + public TagKey[] getHighCardinalityTagKeys() { + return StepHighCardinalityTags.values(); + } + + @Override + public String getPrefix() { + return "spring.batch"; + } + }; + + enum StepLowCardinalityTags implements TagKey { + + /** + * Name of the Spring Batch job. + */ + STEP_NAME { + @Override + public String getKey() { + return "spring.batch.step.name"; + } + }, + + /** + * Type of the Spring Batch job. + */ + STEP_TYPE { + @Override + public String getKey() { + return "spring.batch.step.type"; + } + }, + + /** + * Name of the Spring Batch job. + */ + JOB_NAME { + @Override + public String getKey() { + return "spring.batch.job.name"; + } + }, + + /** + * Step status. + */ + STEP_STATUS { + @Override + public String getKey() { + return "spring.batch.step.status"; + } + } + + } + + enum StepHighCardinalityTags implements TagKey { + + /** + * ID of the Spring Batch execution. + */ + STEP_EXECUTION_ID { + @Override + public String getKey() { + return "spring.batch.step.executionId"; + } + } + + } + +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepTagsProvider.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepTagsProvider.java new file mode 100644 index 0000000000..2d2b458b2b --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/BatchStepTagsProvider.java @@ -0,0 +1,32 @@ +/* + * Copyright 2006-2009 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.step; + +import io.micrometer.core.instrument.observation.Observation; + +/** + * {@link Observation.TagsProvider} for {@link BatchStepContext}. + * + * @author Marcin Grzejszczak + */ +public interface BatchStepTagsProvider extends Observation.TagsProvider { + + @Override + default boolean supportsContext(Observation.Context context) { + return context instanceof BatchStepContext; + } +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/DefaultBatchStepTagsProvider.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/DefaultBatchStepTagsProvider.java new file mode 100644 index 0000000000..6df520b384 --- /dev/null +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/DefaultBatchStepTagsProvider.java @@ -0,0 +1,42 @@ +/* + * Copyright 2011-2018 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.step; + +import io.micrometer.core.instrument.Tags; + +import org.springframework.batch.core.StepExecution; + +/** + * Default {@link BatchStepTagsProvider} implementation. + * + * @author Marcin Grzejszczak + */ +public class DefaultBatchStepTagsProvider implements BatchStepTagsProvider { + @Override + public Tags getLowCardinalityTags(BatchStepContext context) { + StepExecution execution = context.getStepExecution(); + return Tags.of(BatchStepObservation.StepLowCardinalityTags.STEP_NAME.of(execution.getStepName()), + BatchStepObservation.StepLowCardinalityTags.JOB_NAME.of(execution.getJobExecution().getJobInstance().getJobName()), + BatchStepObservation.StepLowCardinalityTags.STEP_STATUS.of(execution.getExitStatus().getExitCode())); + } + + @Override + public Tags getHighCardinalityTags(BatchStepContext context) { + StepExecution execution = context.getStepExecution(); + return Tags.of(BatchStepObservation.StepHighCardinalityTags.STEP_EXECUTION_ID.of(String.valueOf(execution.getId()))); + } + +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/FaultTolerantChunkProcessor.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/FaultTolerantChunkProcessor.java index f188b1b244..09137e55a2 100755 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/FaultTolerantChunkProcessor.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/FaultTolerantChunkProcessor.java @@ -22,7 +22,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.Timer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProcessor.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProcessor.java index 4ec68b4830..d946e39217 100755 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProcessor.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProcessor.java @@ -18,8 +18,8 @@ import java.util.List; -import io.micrometer.api.instrument.Tag; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Timer; import org.springframework.batch.core.StepContribution; import org.springframework.batch.core.StepExecution; diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProvider.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProvider.java index 4c4068ac18..4a14220ae6 100755 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProvider.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/SimpleChunkProvider.java @@ -18,9 +18,9 @@ import java.util.List; -import io.micrometer.api.instrument.Metrics; -import io.micrometer.api.instrument.Tag; -import io.micrometer.api.instrument.Timer; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Timer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.springframework.batch.core.StepContribution; 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 7f202683e8..22cc26955c 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 @@ -31,8 +31,15 @@ import java.util.Date; import java.util.List; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Tags; +import io.micrometer.core.tck.MeterRegistryAssert; +import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.jupiter.api.AfterEach; + import org.springframework.batch.core.BatchStatus; import org.springframework.batch.core.ExitStatus; import org.springframework.batch.core.JobExecution; @@ -201,6 +208,7 @@ public boolean isAllowStartIfComplete() { @Test public void testRunNormally() throws Exception { + Metrics.globalRegistry.withTimerObservationHandler(); step1.setStartLimit(5); step2.setStartLimit(5); job.execute(jobExecution); @@ -211,6 +219,15 @@ public void testRunNormally() throws Exception { assertTrue(step1.passedInJobContext.isEmpty()); assertFalse(step2.passedInJobContext.isEmpty()); + + // Observability + MeterRegistryAssert.assertThat(Metrics.globalRegistry) + .hasTimerWithNameAndTags(BatchJobObservation.BATCH_JOB_OBSERVATION.getName(), Tags.of(Tag.of("error", "none"), Tag.of("spring.batch.job.name", "testJob"), Tag.of("spring.batch.job.status", "COMPLETED"))); + } + + @After + public void cleanup() { + Metrics.globalRegistry.clear(); } @Test diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/step/NonAbstractStepTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/step/NonAbstractStepTests.java index f6b81e7b85..037f5f0db8 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/core/step/NonAbstractStepTests.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/core/step/NonAbstractStepTests.java @@ -23,6 +23,11 @@ import java.util.ArrayList; import java.util.List; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Tags; +import io.micrometer.core.tck.MeterRegistryAssert; +import org.junit.After; import org.junit.Before; import org.junit.Test; import org.springframework.batch.core.BatchStatus; @@ -179,6 +184,7 @@ protected void doExecute(StepExecution stepExecution) throws Exception { */ @Test public void testExecute() throws Exception { + Metrics.globalRegistry.withTimerObservationHandler(); tested.setStepExecutionListeners(new StepExecutionListener[] { listener1, listener2 }); tested.execute(execution); @@ -198,6 +204,15 @@ public void testExecute() throws Exception { repository.saved.containsKey("beforeStep")); assertTrue("Execution context modifications made by listener should be persisted", repository.saved.containsKey("afterStep")); + + // Observability + MeterRegistryAssert.assertThat(Metrics.globalRegistry) + .hasTimerWithNameAndTags(BatchStepObservation.BATCH_STEP_OBSERVATION.getName(), Tags.of(Tag.of("error", "none"), Tag.of("spring.batch.job.name", "jobName"), Tag.of("spring.batch.step.name", "eventTrackingStep"), Tag.of("spring.batch.step.status", "COMPLETED"))); + } + + @After + public void cleanup() { + Metrics.globalRegistry.clear(); } @Test diff --git a/spring-batch-samples/src/main/java/org/springframework/batch/sample/metrics/PrometheusConfiguration.java b/spring-batch-samples/src/main/java/org/springframework/batch/sample/metrics/PrometheusConfiguration.java index 7f15253e36..6c5c226c63 100644 --- a/spring-batch-samples/src/main/java/org/springframework/batch/sample/metrics/PrometheusConfiguration.java +++ b/spring-batch-samples/src/main/java/org/springframework/batch/sample/metrics/PrometheusConfiguration.java @@ -19,7 +19,7 @@ import java.util.Map; import jakarta.annotation.PostConstruct; -import io.micrometer.api.instrument.Metrics; +import io.micrometer.core.instrument.Metrics; import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; import io.prometheus.client.CollectorRegistry; diff --git a/spring-batch-samples/src/test/java/org/springframework/batch/sample/metrics/BatchMetricsTests.java b/spring-batch-samples/src/test/java/org/springframework/batch/sample/metrics/BatchMetricsTests.java index 219380e55c..345537f8de 100644 --- a/spring-batch-samples/src/test/java/org/springframework/batch/sample/metrics/BatchMetricsTests.java +++ b/spring-batch-samples/src/test/java/org/springframework/batch/sample/metrics/BatchMetricsTests.java @@ -25,8 +25,9 @@ import javax.sql.DataSource; -import io.micrometer.api.instrument.Meter; -import io.micrometer.api.instrument.Metrics; +import io.micrometer.core.instrument.Meter; +import io.micrometer.core.instrument.Metrics; +import org.junit.BeforeClass; import org.junit.Test; import org.springframework.batch.core.ExitStatus; @@ -60,6 +61,11 @@ public class BatchMetricsTests { private static final int EXPECTED_SPRING_BATCH_METRICS = 10; + @BeforeClass + public static void setup() { + Metrics.globalRegistry.withTimerObservationHandler(); + } + @Test public void testCalculateDuration() { LocalDateTime startTime = LocalDateTime.now(); diff --git a/spring-batch-test/pom.xml b/spring-batch-test/pom.xml index 3ae9aac70e..b34ee57937 100644 --- a/spring-batch-test/pom.xml +++ b/spring-batch-test/pom.xml @@ -86,5 +86,17 @@ ${slf4j.version} test + + io.micrometer + micrometer-tracing-integration-test + ${micrometer-tracing.version} + test + + + io.micrometer + micrometer-test + ${micrometer.version} + test + diff --git a/spring-batch-test/src/test/java/org/springframework/batch/test/observability/ObservabilitySampleStepTests.java b/spring-batch-test/src/test/java/org/springframework/batch/test/observability/ObservabilitySampleStepTests.java new file mode 100644 index 0000000000..ba81b3b9c3 --- /dev/null +++ b/spring-batch-test/src/test/java/org/springframework/batch/test/observability/ObservabilitySampleStepTests.java @@ -0,0 +1,85 @@ +/* + * Copyright 2008-2021 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.test.observability; + +import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.tck.MeterRegistryAssert; +import io.micrometer.tracing.test.SampleTestRunner; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; + +import org.springframework.batch.core.ExitStatus; +import org.springframework.batch.core.JobExecution; +import org.springframework.batch.core.JobParameters; +import org.springframework.batch.test.JobLauncherTestUtils; +import org.springframework.batch.test.SpringBatchTestJUnit5Tests; +import org.springframework.batch.test.context.SpringBatchTest; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Import; + +import static io.micrometer.tracing.test.simple.SpansAssert.assertThat; + +@SpringBatchTest +public class ObservabilitySampleStepTests extends SampleTestRunner { + + @Autowired + private JobLauncherTestUtils jobLauncherTestUtils; + + private static MeterRegistry registry = Metrics.globalRegistry.withTimerObservationHandler(); + + @Override + protected MeterRegistry getMeterRegistry() { + return registry; + } + + @AfterEach + void clean() { + registry.clear(); + } + + @Override + public SampleTestRunnerConsumer yourCode() { + return (bb, meterRegistry) -> { + // given + JobParameters jobParameters = this.jobLauncherTestUtils.getUniqueJobParameters(); + + // when + JobExecution jobExecution = this.jobLauncherTestUtils.launchJob(jobParameters); + + // then + Assertions.assertEquals(ExitStatus.COMPLETED, jobExecution.getExitStatus()); + + // and + assertThat(bb.getFinishedSpans()) + .haveSameTraceId() + .hasASpanWithName("job") + .hasASpanWithName("step"); + + // and + MeterRegistryAssert.assertThat(meterRegistry) + .hasTimerWithName("spring.batch.job") + .hasTimerWithName("spring.batch.step"); + }; + } + + @Configuration(proxyBeanMethods = false) + @Import(SpringBatchTestJUnit5Tests.JobConfiguration.class) + static class TestConfig { + + } +}