From cc1daad6c5a4735ba21dadfc4becd3ed1ad83657 Mon Sep 17 00:00:00 2001 From: Josh Long Date: Thu, 2 Dec 2010 01:28:09 -0800 Subject: [PATCH 01/16] built a simple test case for BATCH-1509 --- spring-batch-core/pom.xml | 3 +++ .../src/main/java/org/springframework/batch/core/Step.java | 4 ++-- .../batch/core/configuration/xml/AbstractStepParser.java | 6 ++---- .../batch/core/configuration/xml/BeanDefinitionUtils.java | 3 +-- .../batch/core/configuration/xml/CoreNamespaceHandler.java | 4 ++-- .../batch/core/launch/support/SimpleJobOperator.java | 7 ++----- .../batch/core/listener/StepListenerMetaData.java | 3 +-- .../org/springframework/batch/core/step/item/Chunk.java | 6 +++--- .../batch/core/configuration/xml/spring-batch-2.1.xsd | 1 + .../batch/core/listener/StepListenerFactoryBeanTests.java | 2 -- .../springframework/batch/classify/SubclassClassifier.java | 2 +- 11 files changed, 18 insertions(+), 23 deletions(-) diff --git a/spring-batch-core/pom.xml b/spring-batch-core/pom.xml index e3fee1280f..32775ff1ac 100644 --- a/spring-batch-core/pom.xml +++ b/spring-batch-core/pom.xml @@ -40,6 +40,9 @@ hsqldb hsqldb + + com.h2databaseh21.2.132 + commons-io commons-io diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/Step.java b/spring-batch-core/src/main/java/org/springframework/batch/core/Step.java index 014c28875e..df0d610961 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/Step.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/Step.java @@ -17,7 +17,7 @@ /** * Batch domain interface representing the configuration of a step. As with the {@link Job}, a {@link Step} is meant to - * explicitly represent a the configuration of a step by a developer, but also the ability to execute the step. + * explicitly represent the configuration of a step by a developer, but also the ability to execute the step. * * @author Dave Syer * @@ -52,4 +52,4 @@ public interface Step { */ void execute(StepExecution stepExecution) throws JobInterruptedException; -} \ No newline at end of file +} diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java index 28104c834b..6dc59ec23b 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java @@ -137,8 +137,7 @@ protected AbstractBeanDefinition parseStep(Element stepElement, ParserContext pa } - private void parsePartition(Element stepElement, Element partitionElement, AbstractBeanDefinition bd, - ParserContext parserContext, boolean stepUnderspecified) { + private void parsePartition(Element stepElement, Element partitionElement, AbstractBeanDefinition bd, ParserContext parserContext, boolean stepUnderspecified) { bd.setBeanClass(StepParserStepFactoryBean.class); bd.setAttribute("isNamespaceStep", true); @@ -178,8 +177,7 @@ private void parsePartition(Element stepElement, Element partitionElement, Abstr } - private void parseJob(Element stepElement, Element jobElement, AbstractBeanDefinition bd, - ParserContext parserContext, boolean stepUnderspecified) { + private void parseJob(Element stepElement, Element jobElement, AbstractBeanDefinition bd, ParserContext parserContext, boolean stepUnderspecified) { bd.setBeanClass(StepParserStepFactoryBean.class); bd.setAttribute("isNamespaceStep", true); diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/BeanDefinitionUtils.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/BeanDefinitionUtils.java index 3817f58a12..b864f253f5 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/BeanDefinitionUtils.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/BeanDefinitionUtils.java @@ -32,8 +32,7 @@ public class BeanDefinitionUtils { * @return The {@link PropertyValue} for the property of the bean. Search * parent hierarchy if necessary. Return null if none is found. */ - public static PropertyValue getPropertyValue(String beanName, String propertyName, - ConfigurableListableBeanFactory beanFactory) { + public static PropertyValue getPropertyValue(String beanName, String propertyName, ConfigurableListableBeanFactory beanFactory) { return beanFactory.getMergedBeanDefinition(beanName).getPropertyValues().getPropertyValue(propertyName); } diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/CoreNamespaceHandler.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/CoreNamespaceHandler.java index 9373665bb5..2ff043353e 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/CoreNamespaceHandler.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/CoreNamespaceHandler.java @@ -19,14 +19,14 @@ import org.springframework.beans.factory.xml.NamespaceHandlerSupport; /** + * + * * @author Dave Syer * */ public class CoreNamespaceHandler extends NamespaceHandlerSupport { /** - * - * * @see NamespaceHandler#init() */ public void init() { diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/launch/support/SimpleJobOperator.java b/spring-batch-core/src/main/java/org/springframework/batch/core/launch/support/SimpleJobOperator.java index 977c42431d..a16e914f0c 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/launch/support/SimpleJobOperator.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/launch/support/SimpleJobOperator.java @@ -143,9 +143,7 @@ public void setJobLauncher(JobLauncher jobLauncher) { /* * (non-Javadoc) * - * @see - * org.springframework.batch.core.launch.JobOperator#getExecutions(java. - * lang.Long) + * @see org.springframework.batch.core.launch.JobOperator#getExecutions(java.lang.Long) */ public List getExecutions(long instanceId) throws NoSuchJobInstanceException { JobInstance jobInstance = jobExplorer.getJobInstance(instanceId); @@ -251,8 +249,7 @@ public String getSummary(long executionId) throws NoSuchJobExecutionException { * @see * org.springframework.batch.core.launch.JobOperator#resume(java.lang.Long) */ - public Long restart(long executionId) throws JobInstanceAlreadyCompleteException, NoSuchJobExecutionException, - NoSuchJobException, JobRestartException, JobParametersInvalidException { + public Long restart(long executionId) throws JobInstanceAlreadyCompleteException, NoSuchJobExecutionException, NoSuchJobException, JobRestartException, JobParametersInvalidException { logger.info("Checking status of job execution with id=" + executionId); diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/listener/StepListenerMetaData.java b/spring-batch-core/src/main/java/org/springframework/batch/core/listener/StepListenerMetaData.java index 0f17510bc5..518a1f9df8 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/listener/StepListenerMetaData.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/listener/StepListenerMetaData.java @@ -79,8 +79,7 @@ public enum StepListenerMetaData implements ListenerMetaData { private final Class[] paramTypes; private static final Map propertyMap; - StepListenerMetaData(String methodName, String propertyName, Class annotation, - Class listenerInterface, Class... paramTypes) { + StepListenerMetaData(String methodName, String propertyName, Class annotation, Class listenerInterface, Class... paramTypes) { this.methodName = methodName; this.propertyName = propertyName; this.annotation = annotation; diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/Chunk.java b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/Chunk.java index bf0f80d016..936807fc17 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/Chunk.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/step/item/Chunk.java @@ -26,8 +26,8 @@ * Encapsulation of a list of items to be processed and possibly a list of * failed items to be skipped. To mark an item as skipped clients should iterate * over the chunk using the {@link #iterator()} method, and if there is a - * failure call {@link ChunkIterator#remove(Exception)} on the iterator. The - * skipped items are then available through the chunk. + * failure call {@link org.springframework.batch.core.step.item.Chunk.ChunkIterator#remove()} on the iterator. + * The skipped items are then available through the chunk. * * @author Dave Syer * @since 2.0 @@ -246,4 +246,4 @@ public String toString() { } -} \ No newline at end of file +} diff --git a/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd b/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd index 593971a0e0..0b11d697bd 100644 --- a/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd +++ b/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd @@ -453,6 +453,7 @@ + metaDataMap = new HashMap(); - ; metaDataMap.put(AFTER_STEP.getPropertyName(), "destroy"); metaDataMap.put(AFTER_CHUNK.getPropertyName(), "afterChunk"); factoryBean.setMetaDataMap(metaDataMap); diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/classify/SubclassClassifier.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/classify/SubclassClassifier.java index b19c04aae5..a8d76800a9 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/classify/SubclassClassifier.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/classify/SubclassClassifier.java @@ -26,7 +26,7 @@ * A {@link Classifier} for a parameterised object type based on a map. * Classifies objects according to their inheritance relation with the supplied * type map. If the object to be classified is one of the keys of the provided - * map, or is a subclass of one of the keys, then the map entry vale for that + * map, or is a subclass of one of the keys, then the map entry value for that * key is returned. Otherwise returns the default value which is null by * default. * From c1d688b2caba5e6b90206ded8ab545cfcdeb945e Mon Sep 17 00:00:00 2001 From: Josh Long Date: Thu, 2 Dec 2010 02:57:18 -0800 Subject: [PATCH 02/16] working implementation of BATCH-1509 --- .../configuration/xml/AbstractStepParser.java | 40 +++--- .../xml/StepParserStepFactoryBean.java | 123 +++++++++--------- .../configuration/xml/spring-batch-2.1.xsd | 2 +- 3 files changed, 90 insertions(+), 75 deletions(-) diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java index 6dc59ec23b..499e446183 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java @@ -33,11 +33,10 @@ * {@link org.springframework.batch.core.Step} and goes on to (optionally) list * a set of transitions from that step to others with <next on="pattern" * to="stepName"/>. Used by the {@link JobParser}. - * - * @see JobParser - * + * * @author Dave Syer * @author Thomas Risberg + * @see JobParser * @since 2.0 */ public abstract class AbstractStepParser { @@ -60,6 +59,8 @@ public abstract class AbstractStepParser { private static final String STEP_ATTR = "step"; + private static final String STEP_ELE = STEP_ATTR; + private static final String PARTITIONER_ATTR = "partitioner"; private static final String HANDLER_ATTR = "handler"; @@ -75,10 +76,10 @@ public abstract class AbstractStepParser { private static final String JOB_REPO_ATTR = "job-repository"; /** - * @param stepElement The <step/> element + * @param stepElement The <step/> element * @param parserContext * @param jobFactoryRef the reference to the {@link JobParserJobFactoryBean} - * from the enclosing tag. Use 'null' if unknown. + * from the enclosing tag. Use 'null' if unknown. */ protected AbstractBeanDefinition parseStep(Element stepElement, ParserContext parserContext, String jobFactoryRef) { @@ -100,7 +101,7 @@ protected AbstractBeanDefinition parseStep(Element stepElement, ParserContext pa Element partitionElement = DomUtils.getChildElementByTagName(stepElement, PARTITION_ELE); if (partitionElement != null) { boolean stepUnderspecified = CoreNamespaceUtils.isUnderspecified(stepElement); - parsePartition(stepElement, partitionElement, bd, parserContext, stepUnderspecified); + parsePartition(stepElement, partitionElement, bd, parserContext, stepUnderspecified, jobFactoryRef); } Element jobElement = DomUtils.getChildElementByTagName(stepElement, JOB_ELE); @@ -137,7 +138,7 @@ protected AbstractBeanDefinition parseStep(Element stepElement, ParserContext pa } - private void parsePartition(Element stepElement, Element partitionElement, AbstractBeanDefinition bd, ParserContext parserContext, boolean stepUnderspecified) { + private void parsePartition(Element stepElement, Element partitionElement, AbstractBeanDefinition bd, ParserContext parserContext, boolean stepUnderspecified, String jobFactoryRef ) { bd.setBeanClass(StepParserStepFactoryBean.class); bd.setAttribute("isNamespaceStep", true); @@ -145,17 +146,27 @@ private void parsePartition(Element stepElement, Element partitionElement, Abstr String partitionerRef = partitionElement.getAttribute(PARTITIONER_ATTR); String handlerRef = partitionElement.getAttribute(HANDLER_ATTR); - if (!StringUtils.hasText(stepRef)) { - parserContext.getReaderContext().error("You must specify a step", partitionElement); - return; - } if (!StringUtils.hasText(partitionerRef)) { parserContext.getReaderContext().error("You must specify a partitioner", partitionElement); return; } + Element inlineStepElement = DomUtils.getChildElementByTagName(partitionElement, STEP_ELE); + if (inlineStepElement == null && !StringUtils.hasText(stepRef)) { + parserContext.getReaderContext().error("You must specify a step", partitionElement); + return; + } + + MutablePropertyValues propertyValues = bd.getPropertyValues(); - propertyValues.addPropertyValue("step", new RuntimeBeanReference(stepRef)); + + if (StringUtils.hasText(stepRef)) { + propertyValues.addPropertyValue("step", new RuntimeBeanReference(stepRef)); + } else if( inlineStepElement!=null) { + AbstractBeanDefinition stepDefinition = parseStep(inlineStepElement, parserContext, jobFactoryRef); + propertyValues.addPropertyValue("step", stepDefinition ); + } + propertyValues.addPropertyValue("partitioner", new RuntimeBeanReference(partitionerRef)); if (!StringUtils.hasText(handlerRef)) { @@ -170,8 +181,7 @@ private void parsePartition(Element stepElement, Element partitionElement, Abstr propertyValues.addPropertyValue("gridSize", new TypedStringValue(gridSize)); } } - } - else { + } else { propertyValues.addPropertyValue("partitionHandler", new RuntimeBeanReference(handlerRef)); } @@ -205,7 +215,7 @@ private void parseJob(Element stepElement, Element jobElement, AbstractBeanDefin private void parseFlow(Element stepElement, Element flowElement, AbstractBeanDefinition bd, - ParserContext parserContext, boolean stepUnderspecified) { + ParserContext parserContext, boolean stepUnderspecified) { bd.setBeanClass(StepParserStepFactoryBean.class); bd.setAttribute("isNamespaceStep", true); diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java index 864bbaf9a5..23236ca87e 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java @@ -16,10 +16,6 @@ package org.springframework.batch.core.configuration.xml; -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; - import org.springframework.batch.classify.BinaryExceptionClassifier; import org.springframework.batch.core.Job; import org.springframework.batch.core.Step; @@ -66,18 +62,22 @@ import org.springframework.transaction.interceptor.DefaultTransactionAttribute; import org.springframework.util.Assert; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; + /** * This {@link FactoryBean} is used by the batch namespace parser to create * {@link Step} objects. Stores all of the properties that are configurable on * the <step/> (and its inner <tasklet/>). Based on which properties * are configured, the {@link #getObject()} method will delegate to the * appropriate class for generating the {@link Step}. - * + * * @author Dan Garrette - * @since 2.0 * @see SimpleStepFactoryBean * @see FaultTolerantStepFactoryBean * @see TaskletStep + * @since 2.0 */ class StepParserStepFactoryBean implements FactoryBean, BeanNameAware { @@ -194,7 +194,7 @@ class StepParserStepFactoryBean implements FactoryBean, BeanNameAware { /** * Create a {@link Step} from the configuration provided. - * + * * @see FactoryBean#getObject() */ public final Object getObject() throws Exception { @@ -208,34 +208,28 @@ public final Object getObject() throws Exception { configureSimple(fb); configureFaultTolerant(fb); return fb.getObject(); - } - else { + } else { SimpleStepFactoryBean fb = new SimpleStepFactoryBean(); configureSimple(fb); return fb.getObject(); } - } - else if (tasklet != null) { + } else if (tasklet != null) { TaskletStep ts = new TaskletStep(); configureTaskletStep(ts); return ts; - } - else if (flow != null) { + } else if (flow != null) { FlowStep ts = new FlowStep(); configureFlowStep(ts); return ts; - } - else if (job != null) { + } else if (job != null) { JobStep ts = new JobStep(); configureJobStep(ts); return ts; - } - else if (step != null) { + } else if (step != null) { PartitionStep ts = new PartitionStep(); configurePartitionStep(ts); return ts; - } - else { + } else { throw new IllegalStateException("Step [" + name + "] has neither a element nor a 'ref' attribute referencing a Tasklet."); } @@ -274,10 +268,13 @@ private void configurePartitionStep(PartitionStep ts) { Assert.state(partitioner != null, "A Partitioner must be provided for a partition step"); Assert.state(step != null, "A Step must be provided for a partition step"); configureAbstractStep(ts); + + PartitionHandler handler; + if (partitionHandler != null) { + handler = partitionHandler; ts.setPartitionHandler(partitionHandler); - } - else { + } else { TaskExecutorPartitionHandler partitionHandler = new TaskExecutorPartitionHandler(); partitionHandler.setStep(step); if (taskExecutor == null) { @@ -286,7 +283,19 @@ private void configurePartitionStep(PartitionStep ts) { partitionHandler.setGridSize(gridSize); partitionHandler.setTaskExecutor(taskExecutor); ts.setPartitionHandler(partitionHandler); + handler = partitionHandler; } + + if (handler instanceof TaskExecutorPartitionHandler) { + try { + TaskExecutorPartitionHandler taskExecutorPartitionHandler = (TaskExecutorPartitionHandler) handler; + taskExecutorPartitionHandler.setStep(step); + taskExecutorPartitionHandler.afterPropertiesSet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + SimpleStepExecutionSplitter splitter = new SimpleStepExecutionSplitter(jobRepository, step, partitioner); ts.setStepExecutionSplitter(splitter); } @@ -475,18 +484,17 @@ private void validateFaultTolerantSettings() { * Check if a field is present then a second is also. If the * twoWayDependency flag is set then the opposite must also be true: if the * second value is present, the first must also be. - * - * @param dependentName the name of the first field - * @param dependentValue the value of the first field - * @param name the name of the other field (which should be absent if the - * first is present) - * @param value the value of the other field + * + * @param dependentName the name of the first field + * @param dependentValue the value of the first field + * @param name the name of the other field (which should be absent if the + * first is present) + * @param value the value of the other field * @param twoWayDependency true if both depend on each other - * * @throws IllegalArgumentException if eiether condition is violated */ private void validateDependency(String dependentName, Object dependentValue, String name, Object value, - boolean twoWayDependency) { + boolean twoWayDependency) { if (isPresent(dependentValue) && !isPresent(value)) { throw new IllegalArgumentException("The field '" + dependentName + "' is not permitted on the step [" + this.name + "] because there is no '" + name + "'."); @@ -499,7 +507,7 @@ private void validateDependency(String dependentName, Object dependentValue, Str /** * Is the object non-null (or if an Integer, non-zero)? - * + * * @param o an object * @return true if the object has a value */ @@ -538,7 +546,7 @@ public boolean isSingleton() { /** * Set the bean name property, which will become the name of the * {@link Step} when it is created. - * + * * @see org.springframework.beans.factory.BeanNameAware#setBeanName(java.lang.String) */ public void setBeanName(String name) { @@ -562,9 +570,6 @@ public void setFlow(Flow flow) { // Job Attributes // ========================================================= - /** - * @param flow the flow to set - */ public void setJob(Job job) { this.job = job; } @@ -616,7 +621,7 @@ public void setStep(Step step) { /** * Public setter for the flag to indicate that the step should be replayed * on a restart, even if successful the first time. - * + * * @param allowStartIfComplete the shouldAllowStartIfComplete to set */ public void setAllowStartIfComplete(boolean allowStartIfComplete) { @@ -633,7 +638,7 @@ public JobRepository getJobRepository() { /** * Public setter for {@link JobRepository}. - * + * * @param jobRepository */ public void setJobRepository(JobRepository jobRepository) { @@ -642,7 +647,7 @@ public void setJobRepository(JobRepository jobRepository) { /** * The number of times that the step should be allowed to start - * + * * @param startLimit */ public void setStartLimit(int startLimit) { @@ -651,7 +656,7 @@ public void setStartLimit(int startLimit) { /** * A preconfigured {@link Tasklet} to use. - * + * * @param tasklet */ public void setTasklet(Tasklet tasklet) { @@ -680,7 +685,7 @@ public void setTransactionManager(PlatformTransactionManager transactionManager) * The listeners to inject into the {@link Step}. Any instance of * {@link StepListener} can be used, and will then receive callbacks at the * appropriate stage in the step. - * + * * @param listeners an array of listeners */ public void setListeners(StepListener[] listeners) { @@ -690,7 +695,7 @@ public void setListeners(StepListener[] listeners) { /** * Exception classes that may not cause a rollback if encountered in the * right place. - * + * * @param noRollbackExceptionClasses the noRollbackExceptionClasses to set */ public void setNoRollbackExceptionClasses(Collection> noRollbackExceptionClasses) { @@ -724,7 +729,7 @@ public void setPropagation(Propagation propagation) { /** * A backoff policy to be applied to retry process. - * + * * @param backOffPolicy the {@link BackOffPolicy} to set */ public void setBackOffPolicy(BackOffPolicy backOffPolicy) { @@ -734,7 +739,7 @@ public void setBackOffPolicy(BackOffPolicy backOffPolicy) { /** * A retry policy to apply when exceptions occur. If this is specified then * the retry limit and retryable exceptions will be ignored. - * + * * @param retryPolicy the {@link RetryPolicy} to set */ public void setRetryPolicy(RetryPolicy retryPolicy) { @@ -752,7 +757,7 @@ public void setRetryContextCache(RetryContextCache retryContextCache) { * A key generator that can be used to compare items with previously * recorded items in a retry. Only used if the reader is a transactional * queue. - * + * * @param keyGenerator the {@link KeyGenerator} to set */ public void setKeyGenerator(KeyGenerator keyGenerator) { @@ -768,14 +773,14 @@ public void setKeyGenerator(KeyGenerator keyGenerator) { * items than this fail without being skipped or recovered an exception will * be thrown. This is to guard against inadvertent infinite loops generated * by item identity problems.
- * + *

* The default value should be high enough and more for most purposes. To * breach the limit in a single-threaded step typically you have to have * this many failures in a single transaction. Defaults to the value in the * {@link MapRetryContextCache}.
- * + * * @param cacheCapacity the cache capacity to set (greater than 0 else - * ignored) + * ignored) */ public void setCacheCapacity(int cacheCapacity) { this.cacheCapacity = cacheCapacity; @@ -786,7 +791,7 @@ public void setCacheCapacity(int cacheCapacity) { * level. A transaction will be committed when this policy decides to * complete. Defaults to a {@link SimpleCompletionPolicy} with chunk size * equal to the commitInterval property. - * + * * @param chunkCompletionPolicy the chunkCompletionPolicy to set */ public void setChunkCompletionPolicy(CompletionPolicy chunkCompletionPolicy) { @@ -796,7 +801,7 @@ public void setChunkCompletionPolicy(CompletionPolicy chunkCompletionPolicy) { /** * Set the commit interval. Either set this or the chunkCompletionPolicy but * not both. - * + * * @param commitInterval 1 by default */ public void setCommitInterval(int commitInterval) { @@ -807,7 +812,7 @@ public void setCommitInterval(int commitInterval) { * Flag to signal that the reader is transactional (usually a JMS consumer) * so that items are re-presented after a rollback. The default is false and * readers are assumed to be forward-only. - * + * * @param isReaderTransactionalQueue the value of the flag */ public void setIsReaderTransactionalQueue(boolean isReaderTransactionalQueue) { @@ -819,7 +824,7 @@ public void setIsReaderTransactionalQueue(boolean isReaderTransactionalQueue) { * should be called for every item in every transaction. If false then we * can cache the processor results between transactions in the case of a * rollback. - * + * * @param processorTransactional the value to set */ public void setProcessorTransactional(Boolean processorTransactional) { @@ -830,7 +835,7 @@ public void setProcessorTransactional(Boolean processorTransactional) { * Public setter for the retry limit. Each item can be retried up to this * limit. Note this limit includes the initial attempt to process the item, * therefore retryLimit == 1 by default. - * + * * @param retryLimit the retry limit to set, must be greater or equal to 1. */ public void setRetryLimit(int retryLimit) { @@ -843,7 +848,7 @@ public void setRetryLimit(int retryLimit) { * skipped and no exception propagated until the limit is reached. If it is * zero then all exceptions will be propagated from the chunk and cause the * step to abort. - * + * * @param skipLimit the value to set. Default is 0 (never skip). */ public void setSkipLimit(int skipLimit) { @@ -853,7 +858,7 @@ public void setSkipLimit(int skipLimit) { /** * Public setter for a skip policy. If this value is set then the skip limit * and skippable exceptions are ignored. - * + * * @param skipPolicy the {@link SkipPolicy} to set */ public void setSkipPolicy(SkipPolicy skipPolicy) { @@ -863,7 +868,7 @@ public void setSkipPolicy(SkipPolicy skipPolicy) { /** * Public setter for the {@link TaskExecutor}. If this is set, then it will * be used to execute the chunk processing inside the {@link Step}. - * + * * @param taskExecutor the taskExecutor to set */ public void setTaskExecutor(TaskExecutor taskExecutor) { @@ -875,7 +880,7 @@ public void setTaskExecutor(TaskExecutor taskExecutor) { * queued for concurrent processing to prevent thread pools from being * overwhelmed. Defaults to * {@link TaskExecutorRepeatTemplate#DEFAULT_THROTTLE_LIMIT}. - * + * * @param throttleLimit the throttle limit to set. */ public void setThrottleLimit(Integer throttleLimit) { @@ -909,7 +914,7 @@ public void setItemWriter(ItemWriter itemWriter) { /** * Public setter for the {@link RetryListener}s. - * + * * @param retryListeners the {@link RetryListener}s to set */ public void setRetryListeners(RetryListener... retryListeners) { @@ -920,7 +925,7 @@ public void setRetryListeners(RetryListener... retryListeners) { * Public setter for exception classes that when raised won't crash the job * but will result in transaction rollback and the item which handling * caused the exception will be skipped. - * + * * @param exceptionClasses */ public void setSkippableExceptionClasses(Map, Boolean> exceptionClasses) { @@ -929,7 +934,7 @@ public void setSkippableExceptionClasses(Map, Boolean /** * Public setter for exception classes that will retry the item when raised. - * + * * @param retryableExceptionClasses the retryableExceptionClasses to set */ public void setRetryableExceptionClasses(Map, Boolean> retryableExceptionClasses) { @@ -940,7 +945,7 @@ public void setRetryableExceptionClasses(Map, Boolean * The streams to inject into the {@link Step}. Any instance of * {@link ItemStream} can be used, and will then receive callbacks at the * appropriate stage in the step. - * + * * @param streams an array of listeners */ public void setStreams(ItemStream[] streams) { diff --git a/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd b/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd index 0b11d697bd..090753d961 100644 --- a/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd +++ b/spring-batch-core/src/main/resources/org/springframework/batch/core/configuration/xml/spring-batch-2.1.xsd @@ -453,7 +453,7 @@ - + Date: Thu, 2 Dec 2010 03:01:28 -0800 Subject: [PATCH 03/16] working implementation of BATCH-1509 --- .../core/partition/support/TaskExecutorPartitionHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/partition/support/TaskExecutorPartitionHandler.java b/spring-batch-core/src/main/java/org/springframework/batch/core/partition/support/TaskExecutorPartitionHandler.java index 9982167519..ab74b10449 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/partition/support/TaskExecutorPartitionHandler.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/partition/support/TaskExecutorPartitionHandler.java @@ -54,7 +54,7 @@ public class TaskExecutorPartitionHandler implements PartitionHandler, Initializ private Step step; public void afterPropertiesSet() throws Exception { - Assert.notNull(step, "A Step must be provided."); +// Assert.notNull(step, "A Step must be provided."); } /** From b7ab8b84310874f433f619e0df6d054ed80bdcd4 Mon Sep 17 00:00:00 2001 From: Josh Long Date: Thu, 2 Dec 2010 03:02:07 -0800 Subject: [PATCH 04/16] working implementation of BATCH-1509 --- .../batch/sandbox/Customer.java | 44 ++++++++++++++ .../batch/sandbox/CustomerRowMapper.java | 16 +++++ .../batch/sandbox/LoggingItemWriter.java | 18 ++++++ .../batch/sandbox/PartionedStepClient.java | 30 ++++++++++ .../org/springframework/batch/sandbox/ps1.xml | 59 +++++++++++++++++++ 5 files changed, 167 insertions(+) create mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java create mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java create mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java create mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java create mode 100644 spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java new file mode 100644 index 0000000000..d7ed40f04f --- /dev/null +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java @@ -0,0 +1,44 @@ +package org.springframework.batch.sandbox; + + +/** + * @author Josh Long + */ +public class Customer { + + private long id; + + private String firstName; + + private String lastName; + + private String email; + + public Customer(long id, String firstName, String lastName, String email) { + this.id = id; + this.firstName = firstName; + this.lastName = lastName; + this.email = email; + } + + public String getEmail() { + return email; + } + + public long getId() { + return id; + } + + public String getFirstName() { + return firstName; + } + + public String getLastName() { + return lastName; + } + + @Override + public String toString() { + return "Customer{" + "id=" + id + ", firstName='" + firstName + '\'' + ", lastName='" + lastName + '\'' + ", email='" + email + '\'' + '}'; + } +} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java new file mode 100644 index 0000000000..54506d5560 --- /dev/null +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java @@ -0,0 +1,16 @@ +package org.springframework.batch.sandbox; + +import org.springframework.jdbc.core.RowMapper; + +import java.sql.ResultSet; +import java.sql.SQLException; + + +/** + * @author Josh Long + */ +public class CustomerRowMapper implements RowMapper { + public Object mapRow(ResultSet resultSet, int i) throws SQLException { + return new Customer( resultSet.getLong( "ID") , resultSet.getString("FIRST_NAME") , resultSet.getString( "LAST_NAME") , resultSet.getString( "EMAIL")) ; + } +} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java new file mode 100644 index 0000000000..6a26e2de4c --- /dev/null +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java @@ -0,0 +1,18 @@ +package org.springframework.batch.sandbox; + +import org.springframework.batch.item.ItemWriter; + +import java.util.List; + + +public class LoggingItemWriter implements ItemWriter { + public void write(List items) throws Exception { + + System.out.println(); + + for( Customer c : items) + System.out.println( "customer = " + c.toString()); + + System.out.println( "----------------------------------------------------------------------------------------" ); + } +} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java new file mode 100644 index 0000000000..378af6a1aa --- /dev/null +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java @@ -0,0 +1,30 @@ +package org.springframework.batch.sandbox; + +import org.springframework.batch.core.Job; +import org.springframework.batch.core.JobExecution; +import org.springframework.batch.core.JobParameters; +import org.springframework.batch.core.JobParametersBuilder; +import org.springframework.batch.core.launch.JobLauncher; +import org.springframework.context.support.ClassPathXmlApplicationContext; + +import java.util.UUID; + +/** + * @author Josh Long + * @see {@link org.springframework.batch.core.partition.support.PartitionStep} + */ +public class PartionedStepClient { + public static void main (String [] args ) throws Throwable { + + + ClassPathXmlApplicationContext cax = new ClassPathXmlApplicationContext ("org/springframework/batch/sandbox/ps1.xml"); + + JobLauncher jobLauncher = (JobLauncher) cax.getBean("jobLauncher"); + Job job = (Job) cax.getBean("partitionedJob") ; + + + JobParameters parms = new JobParametersBuilder().addString("uid", UUID.randomUUID().toString()).toJobParameters(); + JobExecution execution = jobLauncher.run( job, parms); + + } +} diff --git a/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml b/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml new file mode 100644 index 0000000000..54dc0fb03f --- /dev/null +++ b/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml @@ -0,0 +1,59 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From b71e6862fadbf412bdba889d8a6e7de6b45267c4 Mon Sep 17 00:00:00 2001 From: Josh Long Date: Thu, 2 Dec 2010 03:11:10 -0800 Subject: [PATCH 05/16] working implementation of BATCH-1509 --- .../springframework/batch/sandbox/Customer.java | 15 +++++++++++++++ .../batch/sandbox/CustomerRowMapper.java | 15 +++++++++++++++ .../batch/sandbox/LoggingItemWriter.java | 15 +++++++++++++++ .../batch/sandbox/PartionedStepClient.java | 15 +++++++++++++++ .../org/springframework/batch/sandbox/ps1.xml | 16 ++++++++++++++++ 5 files changed, 76 insertions(+) diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java index d7ed40f04f..0e2d58c4ea 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java @@ -1,3 +1,18 @@ +/* + * Copyright 2006-2007 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.springframework.batch.sandbox; diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java index 54506d5560..877b3ed69d 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java @@ -1,3 +1,18 @@ +/* + * Copyright 2006-2007 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.springframework.batch.sandbox; import org.springframework.jdbc.core.RowMapper; diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java index 6a26e2de4c..dce7224071 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java @@ -1,3 +1,18 @@ +/* + * Copyright 2006-2007 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.springframework.batch.sandbox; import org.springframework.batch.item.ItemWriter; diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java index 378af6a1aa..02ee335ecf 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java @@ -1,3 +1,18 @@ +/* + * Copyright 2006-2007 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.springframework.batch.sandbox; import org.springframework.batch.core.Job; diff --git a/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml b/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml index 54dc0fb03f..32d5647d92 100644 --- a/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml +++ b/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml @@ -1,4 +1,20 @@ + Date: Thu, 2 Dec 2010 03:11:37 -0800 Subject: [PATCH 06/16] working implementation of BATCH-1509 --- .../batch/core/configuration/xml/AbstractStepParser.java | 1 + 1 file changed, 1 insertion(+) diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java index 499e446183..eb4d7aa5db 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/AbstractStepParser.java @@ -36,6 +36,7 @@ * * @author Dave Syer * @author Thomas Risberg + * @author Josh Long * @see JobParser * @since 2.0 */ From f7ae78ecb2359754f1dbb8042a93a585926b6310 Mon Sep 17 00:00:00 2001 From: Dave Syer Date: Fri, 26 Nov 2010 19:01:15 +0800 Subject: [PATCH 07/16] BATCH-1659: add try/catch to file creation --- .../batch/item/util/FileUtils.java | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java index 8d8de3f649..5be6d29231 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java @@ -62,17 +62,15 @@ public static void setUpOutputFile(File file, boolean restarted, boolean overwri throw new ItemStreamException("File already exists: [" + file.getAbsolutePath() + "]"); } if (!file.delete()) { - throw new IOException("Could not delete file: "+file); + throw new IOException("Could not delete file: " + file); } } if (file.getParent() != null) { new File(file.getParent()).mkdirs(); } - file.createNewFile(); - if (!file.exists()) { - throw new ItemStreamException("Output file was not created: [" + file.getAbsolutePath() - + "]"); + if (!createNewFile(file)) { + throw new ItemStreamException("Output file was not created: [" + file.getAbsolutePath() + "]"); } } } @@ -84,4 +82,32 @@ public static void setUpOutputFile(File file, boolean restarted, boolean overwri throw new ItemStreamException("File is not writable: [" + file.getAbsolutePath() + "]"); } } + + /** + * Create a new file if it doesn't already exist. + * + * @param file the file to create on the filesystem + */ + public static boolean createNewFile(File file) throws IOException { + + if (file.exists()) { + return false; + } + + try { + return file.createNewFile(); + } + catch (IOException e) { + // On some filesystems you can get an exception here even though the + // files was successfully created + if (file.exists()) { + return true; + } + else { + throw e; + } + } + + } + } From 810bdb70c64e4706a14b54e5e366e79b3b512656 Mon Sep 17 00:00:00 2001 From: Dave Syer Date: Fri, 26 Nov 2010 21:35:31 +0800 Subject: [PATCH 08/16] Upgrade Spring 3 for integration test --- spring-batch-parent/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spring-batch-parent/pom.xml b/spring-batch-parent/pom.xml index 20b4a817c5..046e0029af 100644 --- a/spring-batch-parent/pom.xml +++ b/spring-batch-parent/pom.xml @@ -32,8 +32,8 @@ spring3 - 3.0.3.BUILD-SNAPSHOT - 4.7 + 3.0.5.RELEASE + 4.8.2 From ec54bf42bbe0c26e00fd28f54405ebc443df4497 Mon Sep 17 00:00:00 2001 From: Robert Kasanicky Date: Mon, 29 Nov 2010 04:49:12 +0800 Subject: [PATCH 09/16] fix eclipse buildpath error --- spring-batch-infrastructure-tests/.classpath | 1 - 1 file changed, 1 deletion(-) diff --git a/spring-batch-infrastructure-tests/.classpath b/spring-batch-infrastructure-tests/.classpath index 2daddec399..3cb90946ef 100644 --- a/spring-batch-infrastructure-tests/.classpath +++ b/spring-batch-infrastructure-tests/.classpath @@ -1,7 +1,6 @@ - From 30f7ad9b45433ec1bee1c57f92337ee096b0643f Mon Sep 17 00:00:00 2001 From: Robert Kasanicky Date: Sun, 5 Dec 2010 05:40:09 +0800 Subject: [PATCH 10/16] RESOLVED - BATCH-1661: FlatFileItemReader always logs as ERROR non data lines even though the row should be skipped --- .../batch/item/file/FlatFileItemReader.java | 55 +++++------ .../item/file/mapping/DefaultLineMapper.java | 17 +--- .../item/file/mapping/JsonLineMapper.java | 21 ++-- .../item/file/FlatFileItemReaderTests.java | 99 ++++++++++++------- .../file/mapping/DefaultLineMapperTests.java | 57 +---------- .../file/mapping/JsonLineMapperTests.java | 7 +- 6 files changed, 108 insertions(+), 148 deletions(-) diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/FlatFileItemReader.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/FlatFileItemReader.java index b772a2aaed..4ca4c567c6 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/FlatFileItemReader.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/FlatFileItemReader.java @@ -34,10 +34,10 @@ import org.springframework.util.StringUtils; /** - * Restartable {@link ItemReader} that reads lines from input - * {@link #setResource(Resource)}. Line is defined by the - * {@link #setRecordSeparatorPolicy(RecordSeparatorPolicy)} and mapped to item - * using {@link #setLineMapper(LineMapper)}. + * Restartable {@link ItemReader} that reads lines from input {@link #setResource(Resource)}. Line is defined by the + * {@link #setRecordSeparatorPolicy(RecordSeparatorPolicy)} and mapped to item using {@link #setLineMapper(LineMapper)}. + * If an exception is thrown during line mapping it is rethrown as {@link FlatFileParseException} adding information + * about the problematic line and its line number. * * @author Robert Kasanicky */ @@ -79,8 +79,7 @@ public FlatFileItemReader() { /** * In strict mode the reader will throw an exception on - * {@link #open(org.springframework.batch.item.ExecutionContext)} if the - * input resource does not exist. + * {@link #open(org.springframework.batch.item.ExecutionContext)} if the input resource does not exist. * @param strict false by default */ public void setStrict(boolean strict) { @@ -88,18 +87,15 @@ public void setStrict(boolean strict) { } /** - * @param skippedLinesCallback will be called for each one of the initial - * skipped lines before any items are read. + * @param skippedLinesCallback will be called for each one of the initial skipped lines before any items are read. */ public void setSkippedLinesCallback(LineCallbackHandler skippedLinesCallback) { this.skippedLinesCallback = skippedLinesCallback; } /** - * Public setter for the number of lines to skip at the start of a file. Can - * be used if the file contains a header without useful (column name) - * information, and without a comment delimiter at the beginning of the - * lines. + * Public setter for the number of lines to skip at the start of a file. Can be used if the file contains a header + * without useful (column name) information, and without a comment delimiter at the beginning of the lines. * * @param linesToSkip the number of lines to skip */ @@ -116,21 +112,18 @@ public void setLineMapper(LineMapper lineMapper) { } /** - * Setter for the encoding for this input source. Default value is - * {@link #DEFAULT_CHARSET}. + * Setter for the encoding for this input source. Default value is {@link #DEFAULT_CHARSET}. * - * @param encoding a properties object which possibly contains the encoding - * for this input file; + * @param encoding a properties object which possibly contains the encoding for this input file; */ public void setEncoding(String encoding) { this.encoding = encoding; } /** - * Factory for the {@link BufferedReader} that will be used to extract lines - * from the file. The default is fine for plain text files, but this is a - * useful strategy for binary files where the standard BufferedReaader from - * java.io is limiting. + * Factory for the {@link BufferedReader} that will be used to extract lines from the file. The default is fine for + * plain text files, but this is a useful strategy for binary files where the standard BufferedReaader from java.io + * is limiting. * * @param bufferedReaderFactory the bufferedReaderFactory to set */ @@ -139,8 +132,8 @@ public void setBufferedReaderFactory(BufferedReaderFactory bufferedReaderFactory } /** - * Setter for comment prefixes. Can be used to ignore header lines as well - * by using e.g. the first couple of column names as a prefix. + * Setter for comment prefixes. Can be used to ignore header lines as well by using e.g. the first couple of column + * names as a prefix. * * @param comments an array of comment line prefixes. */ @@ -157,9 +150,8 @@ public void setResource(Resource resource) { } /** - * Public setter for the recordSeparatorPolicy. Used to determine where the - * line endings are and do things like continue over a line ending if inside - * a quoted string. + * Public setter for the recordSeparatorPolicy. Used to determine where the line endings are and do things like + * continue over a line ending if inside a quoted string. * * @param recordSeparatorPolicy the recordSeparatorPolicy to set */ @@ -169,8 +161,7 @@ public void setRecordSeparatorPolicy(RecordSeparatorPolicy recordSeparatorPolicy /** * @return string corresponding to logical record according to - * {@link #setRecordSeparatorPolicy(RecordSeparatorPolicy)} (might span - * multiple lines in file). + * {@link #setRecordSeparatorPolicy(RecordSeparatorPolicy)} (might span multiple lines in file). */ @Override protected T doRead() throws Exception { @@ -188,15 +179,14 @@ protected T doRead() throws Exception { return lineMapper.mapLine(line, lineCount); } catch (Exception ex) { - logger.error("Parsing error at line: " + lineCount + " in resource=" + resource.getDescription() - + ", input=[" + line + "]", ex); - throw ex; + throw new FlatFileParseException("Parsing error at line: " + lineCount + " in resource=[" + + resource.getDescription() + "], input=[" + line + "]", ex, line, lineCount); } } } /** - * @return next line (skip comments). + * @return next line (skip comments).getCurrentResource */ private String readLine() { @@ -226,7 +216,8 @@ private String readLine() { // Prevent IOException from recurring indefinitely // if client keeps catching and re-calling noInput = true; - throw new NonTransientFlatFileException("Unable to read from resource: [" + resource + "]", e, line, lineCount); + throw new NonTransientFlatFileException("Unable to read from resource: [" + resource + "]", e, line, + lineCount); } return line; } diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/DefaultLineMapper.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/DefaultLineMapper.java index 1b199959fc..4a21f1d86c 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/DefaultLineMapper.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/DefaultLineMapper.java @@ -16,7 +16,6 @@ package org.springframework.batch.item.file.mapping; -import org.springframework.batch.item.file.FlatFileParseException; import org.springframework.batch.item.file.LineMapper; import org.springframework.batch.item.file.transform.FieldSet; import org.springframework.batch.item.file.transform.LineTokenizer; @@ -24,11 +23,9 @@ import org.springframework.util.Assert; /** - * Two-phase {@link LineMapper} implementation consisting of tokenization of the - * line into {@link FieldSet} followed by mapping to item. By default, any exceptions - * thrown by either delegates will be wrapped in a {@link FlatFileParseException} before - * being rethrown. If finer grained control of exceptions is needed, the {@link LineMapper} - * interface should be implemented directly. + * Two-phase {@link LineMapper} implementation consisting of tokenization of the line into {@link FieldSet} followed by + * mapping to item. If finer grained control of exceptions is needed, the {@link LineMapper} interface should be + * implemented directly. * * @author Robert Kasanicky * @author Lucas Ward @@ -42,13 +39,7 @@ public class DefaultLineMapper implements LineMapper, InitializingBean { private FieldSetMapper fieldSetMapper; public T mapLine(String line, int lineNumber) throws Exception { - try{ - return fieldSetMapper.mapFieldSet(tokenizer.tokenize(line)); - } - catch(Exception ex){ - throw new FlatFileParseException("Parsing error at line: " + lineNumber + - ", input=[" + line + "]", ex, line, lineNumber); - } + return fieldSetMapper.mapFieldSet(tokenizer.tokenize(line)); } public void setLineTokenizer(LineTokenizer tokenizer) { diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/JsonLineMapper.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/JsonLineMapper.java index 602bb3ec54..41b0f4d8f0 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/JsonLineMapper.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/mapping/JsonLineMapper.java @@ -4,13 +4,11 @@ import org.codehaus.jackson.JsonParser; import org.codehaus.jackson.map.MappingJsonFactory; -import org.springframework.batch.item.file.FlatFileParseException; import org.springframework.batch.item.file.LineMapper; /** - * Interpret a line as a Json object and parse it up to a Map. The line should - * be a standard Json object, starting with "{" and ending with "}" and composed - * of name:value pairs separated by commas. Whitespace is ignored, + * Interpret a line as a Json object and parse it up to a Map. The line should be a standard Json object, starting with + * "{" and ending with "}" and composed of name:value pairs separated by commas. Whitespace is ignored, * e.g. * *

@@ -31,21 +29,16 @@ public class JsonLineMapper implements LineMapper> {
 	private MappingJsonFactory factory = new MappingJsonFactory();
 
 	/**
-	 * Interpret the line as a Json object and create a Map from it.  
+	 * Interpret the line as a Json object and create a Map from it.
 	 * 
 	 * @see LineMapper#mapLine(String, int)
 	 */
 	public Map mapLine(String line, int lineNumber) throws Exception {
 		Map result;
-		try {
-			JsonParser parser = factory.createJsonParser(line);
-			@SuppressWarnings("unchecked")
-			Map token = parser.readValueAs(Map.class);
-			result = token;
-		}
-		catch (Exception e) {
-			throw new FlatFileParseException("Cannot parse line to JSON", e, line, lineNumber);
-		}
+		JsonParser parser = factory.createJsonParser(line);
+		@SuppressWarnings("unchecked")
+		Map token = parser.readValueAs(Map.class);
+		result = token;
 		return result;
 	}
 
diff --git a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java
index 9c5b93bbab..50f42c5d3e 100644
--- a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java
+++ b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java
@@ -193,8 +193,7 @@ public void testRestartWithSkippedLines() throws Exception {
 		// close input
 		reader.close();
 
-		reader
-				.setResource(getInputResource("header\nignoreme\ntestLine1\ntestLine2\ntestLine3\ntestLine4\ntestLine5\ntestLine6"));
+		reader.setResource(getInputResource("header\nignoreme\ntestLine1\ntestLine2\ntestLine3\ntestLine4\ntestLine5\ntestLine6"));
 
 		// init for restart
 		reader.open(executionContext);
@@ -302,37 +301,37 @@ public void testNonExistentResource() throws Exception {
 		reader.close();
 	}
 
-    @Test
-    public void testOpenBadIOInput() throws Exception {
-
-        reader.setResource(new AbstractResource() {
-            public String getDescription() {
-                return null;
-            }
-
-            public InputStream getInputStream() throws IOException {
-                throw new IOException();
-            }
-
-            public boolean exists() {
-                return true;
-            }
-        });
-
-        try {
-            reader.open(executionContext);
-            fail();
-        }
-        catch (ItemStreamException ex) {
-            // expected
-        }
-        
-        // read() should then return a null
-        assertNull(reader.read());
-        reader.close();
-
-    }
-	
+	@Test
+	public void testOpenBadIOInput() throws Exception {
+
+		reader.setResource(new AbstractResource() {
+			public String getDescription() {
+				return null;
+			}
+
+			public InputStream getInputStream() throws IOException {
+				throw new IOException();
+			}
+
+			public boolean exists() {
+				return true;
+			}
+		});
+
+		try {
+			reader.open(executionContext);
+			fail();
+		}
+		catch (ItemStreamException ex) {
+			// expected
+		}
+
+		// read() should then return a null
+		assertNull(reader.read());
+		reader.close();
+
+	}
+
 	@Test
 	public void testDirectoryResource() throws Exception {
 
@@ -381,6 +380,40 @@ public void testStrictness() throws Exception {
 		reader.open(executionContext);
 	}
 
+	/**
+	 * Exceptions from {@link LineMapper} are wrapped as {@link FlatFileParseException} containing contextual info about
+	 * the problematic line and its line number.
+	 */
+	@Test
+	public void testMappingExceptionWrapping() throws Exception {
+		LineMapper exceptionLineMapper = new LineMapper() {
+
+			@Override
+			public String mapLine(String line, int lineNumber) throws Exception {
+				if (lineNumber == 2) {
+					throw new Exception("Couldn't map line 2");
+				}
+				return line;
+			}
+		};
+		reader.setLineMapper(exceptionLineMapper);
+		reader.afterPropertiesSet();
+
+		reader.open(executionContext);
+		assertNotNull(reader.read());
+
+		try {
+			reader.read();
+			fail();
+		}
+		catch (FlatFileParseException expected) {
+			assertEquals(2, expected.getLineNumber());
+			assertEquals("testLine2", expected.getInput());
+			assertEquals("Couldn't map line 2", expected.getCause().getMessage());
+			assertEquals("Parsing error at line: 2 in resource=[resource loaded from byte array], input=[testLine2]", expected.getMessage());
+		}
+	}
+
 	private Resource getInputResource(String input) {
 		return new ByteArrayResource(input.getBytes());
 	}
diff --git a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/DefaultLineMapperTests.java b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/DefaultLineMapperTests.java
index 1ce6f20442..7e633e6b3a 100644
--- a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/DefaultLineMapperTests.java
+++ b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/DefaultLineMapperTests.java
@@ -1,10 +1,12 @@
 package org.springframework.batch.item.file.mapping;
 
-import static org.easymock.EasyMock.*;
-import static org.junit.Assert.*;
+import static org.easymock.EasyMock.createStrictMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.Assert.assertSame;
 
 import org.junit.Test;
-import org.springframework.batch.item.file.FlatFileParseException;
 import org.springframework.batch.item.file.transform.DefaultFieldSet;
 import org.springframework.batch.item.file.transform.DelimitedLineTokenizer;
 import org.springframework.batch.item.file.transform.FieldSet;
@@ -54,53 +56,4 @@ public void testMapping() throws Exception {
 		
 	}
 	
-	@Test
-	public void testTokenizerException() throws Exception {
-		final String line = "TEST";
-		
-		LineTokenizer tokenizer = createStrictMock(LineTokenizer.class);
-		expect(tokenizer.tokenize(line)).andThrow(new RuntimeException());
-		replay(tokenizer);
-		
-		@SuppressWarnings("unchecked")
-		FieldSetMapper fsMapper = createStrictMock(FieldSetMapper.class);
-		
-		tested.setLineTokenizer(tokenizer);
-		tested.setFieldSetMapper(fsMapper);
-		
-		try{
-			tested.mapLine(line, 1);
-		}
-		catch(FlatFileParseException ex){
-			assertEquals(ex.getLineNumber(), 1);
-			assertEquals(ex.getInput(), line);
-		}
-	}
-	
-	@Test
-	public void testMapperException() throws Exception {
-		final String line = "TEST";
-		final FieldSet fs = new DefaultFieldSet(new String[]{"token1", "token2"});
-		
-		LineTokenizer tokenizer = createStrictMock(LineTokenizer.class);
-		expect(tokenizer.tokenize(line)).andReturn(fs);
-		replay(tokenizer);
-		
-		@SuppressWarnings("unchecked")
-		FieldSetMapper fsMapper = createStrictMock(FieldSetMapper.class);
-		expect(fsMapper.mapFieldSet(fs)).andThrow(new RuntimeException());
-		replay(fsMapper);
-		
-		tested.setLineTokenizer(tokenizer);
-		tested.setFieldSetMapper(fsMapper);
-		
-		try{
-			tested.mapLine(line, 1);
-		}
-		catch(FlatFileParseException ex){
-			assertEquals(ex.getLineNumber(), 1);
-			assertEquals(ex.getInput(), line);
-		}
-		
-	}
 }
diff --git a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/JsonLineMapperTests.java b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/JsonLineMapperTests.java
index 4acafd5fe7..2c4ac6e691 100644
--- a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/JsonLineMapperTests.java
+++ b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/mapping/JsonLineMapperTests.java
@@ -1,12 +1,11 @@
 package org.springframework.batch.item.file.mapping;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.util.Map;
 
+import org.codehaus.jackson.JsonParseException;
 import org.junit.Test;
-import org.springframework.batch.item.file.FlatFileParseException;
-import org.springframework.batch.item.file.mapping.JsonLineMapper;
 
 public class JsonLineMapperTests {
 	
@@ -26,7 +25,7 @@ public void testMapNested() throws Exception {
 		assertEquals(2, ((Map) map.get("bar")).get("foo"));
 	}
 
-	@Test(expected=FlatFileParseException.class)
+	@Test(expected=JsonParseException.class)
 	public void testMappingError() throws Exception {
 		Map map = mapper.mapLine("{\"foo\": 1", 1);
 		assertEquals(1, map.get("foo"));

From 7fb5812b3410ec78d9002e7b41dad8a86e98c39d Mon Sep 17 00:00:00 2001
From: Dave Syer 
Date: Tue, 7 Dec 2010 01:31:29 +0800
Subject: [PATCH 11/16] Add notes to user guide on kill -9

---
 src/site/docbook/reference/job.xml | 361 ++++++++++++++++-------------
 1 file changed, 203 insertions(+), 158 deletions(-)

diff --git a/src/site/docbook/reference/job.xml b/src/site/docbook/reference/job.xml
index dcff2a4982..dd4c891e05 100644
--- a/src/site/docbook/reference/job.xml
+++ b/src/site/docbook/reference/job.xml
@@ -4,9 +4,9 @@
 
   Configuring and Running a Job
 
-  In the domain section , the
-  overall architecture design was discussed, using the following
-  diagram as a guide:
+  In the domain section , the overall
+  architecture design was discussed, using the following diagram as a
+  guide:
 
   
     
@@ -38,11 +38,11 @@
     required dependencies: a name, JobRepository , and
     a list of Steps.
 
-    <job id="footballJob">
-    <step id="playerload"          parent="s1" next="gameLoad"/>
-    <step id="gameLoad"            parent="s2" next="playerSummarization"/>
-    <step id="playerSummarization" parent="s3"/>
-</job>
+    
+    
+    
+    
+]]>
 
     The examples here use a parent bean definition to create the steps;
     see the section on step configuration
@@ -50,11 +50,11 @@
     defaults to referencing a repository with an id of 'jobRepository', which
     is a sensible default. However, this can be overridden explicitly:
 
-    <job id="footballJob" job-repository="specialRepository">
-    <step id="playerload"          parent="s1" next="gameLoad"/>
-    <step id="gameLoad"            parent="s3" next="playerSummarization"/>
-    <step id="playerSummarization" parent="s3"/>
-</job>
+    job-repository="specialRepository"
+    
+    
+    
+]]>
 
     In addition to steps a job configuration can contain other elements
     that help with parallelisation (<split/>),
@@ -78,16 +78,16 @@
       be run as part of a new JobInstance, then the
       restartable property may be set to 'false':
 
-      <job id="footballJob" restartable="false">
+      restartable="false"
     ...
-</job>
+]]>
 
       To phrase it another way, setting restartable to false means "this
       Job does not support being started again". Restarting a Job that is not
       restartable will cause a JobRestartException to
       be thrown:
 
-      Job job = new SimpleJob();
+      
+}]]>
 
       This snippet of JUnit code shows how attempting to create a
       JobExecution the first time for a non restartable
@@ -118,40 +118,40 @@ catch (JobRestartException e) {
       SimpleJob allows for this by calling a
       JobListener at the appropriate time:
 
-      public interface JobExecutionListener {
+      
+}]]>
 
       JobListeners can be added to a
       SimpleJob via the listeners element on the
       job:
 
-      <job id="footballJob">
-    <step id="playerload"          parent="s1" next="gameLoad"/>
-    <step id="gameLoad"            parent="s2" next="playerSummarization"/>
-    <step id="playerSummarization" parent="s3"/>
-    <listeners>
+      
+    
+    
+    
+]]>    <listeners>
         <listener ref="sampleListener"/>
     </listeners>
-</job>
+]]>
 
       It should be noted that afterJob will be
       called regardless of the success or failure of the
       Job. If success or failure needs to be determined
       it can be obtained from the JobExecution:
 
-      public void afterJob(JobExecution jobExecution){
+      
+}]]>
 
       The annotations corresponding to this interface are:
 
@@ -184,19 +184,19 @@ catch (JobRestartException e) {
       Job with two listeners and one
       Step, "step1".
 
-      <job id="baseJob" abstract="true">
-    <listeners>
-        <listener ref="listenerOne"/>
-    <listeners>
-</job>
+      
+    
+        
+    
+
 
-<job id="job1" parent="baseJob3">
-    <step id="step1" parent="standaloneStep"/>
+
+    
 
-    <listeners merge="true">
-        <listener ref="listenerTwo"/>
-    <listeners>
-</job>
+    
+        
+    
+]]>
 
       Please see the section on Inheriting from a Parent Step
@@ -216,10 +216,10 @@ catch (JobRestartException e) {
       of a validator is supported through the XML namespace through a child
       element of the job, e.g>:
 
-      <job id="job1" parent="baseJob3">
-    <step id="step1" parent="standaloneStep"/>
-    <validator ref="paremetersValidator"/>
-</job>
+      
+    
+    
+]]>
 
       The validator can be specified as a reference (as above) or as a
       nested bean definition in the beans namespace.
@@ -227,12 +227,17 @@ catch (JobRestartException e) {
   
 
   
+ + Configuring a JobRepository - As described in earlier, the JobRepository is - used for basic CRUD operations of the various persisted domain objects - within Spring Batch, such as JobExecution and + + + As described in earlier, the + JobRepository + is used for basic CRUD operations of the various persisted + domain objects within Spring Batch, such as + JobExecution and StepExecution. It is required by many of the major framework features, such as the JobLauncher, Job, and Step. The batch @@ -241,22 +246,26 @@ catch (JobRestartException e) { collaborators. However, there are still a few configuration options available: - <job-repository id="jobRepository" + + + +/>]]> + + None of the configuration options listed above are required except the id. If they are not set, the defaults shown above will be used. They - are shown above for awareness purposes. The max-varchar-length defaults to 2500, which is the length of the long - VARCHAR - columns in the sample schema scripts used - to store things like exit code descriptions. If you don't modify the schema - and you don't use multi-bye characters you shouldn't need to change it. + are shown above for awareness purposes. The + max-varchar-length defaults to 2500, which is the + length of the long VARCHAR columns in the sample schema scripts + + used to store things like exit code descriptions. If you don't modify the schema and you don't use multi-bye characters you shouldn't need to change it.
Transaction Configuration for the JobRepository @@ -277,24 +286,28 @@ catch (JobRestartException e) { that the SERIALIZED will cause problems, as long as the database platform supports it. However, this can be overridden: - <job-repository id="jobRepository" - isolation-level-for-create="REPEATABLE_READ" /> + + isolation-level-for-create="REPEATABLE_READ"]]> + If the namespace or factory beans aren't used then it is also essential to configure the transactional behavior of the repository using AOP: - <aop:config> - <aop:advisor - pointcut="execution(* org.springframework.batch.core..*Repository+.*(..))"/> - <advice-ref="txAdvice" /> -</aop:config> + + + + + -<tx:advice id="txAdvice" transaction-manager="transactionManager"> - <tx:attributes> - <tx:method name="*" /> - </tx:attributes> -</tx:advice> + + + + +]]> + This fragment can be used as is, with almost no changes. Remember also to include the appropriate namespace declarations and to make sure @@ -302,6 +315,8 @@ catch (JobRestartException e) { classpath.
+ +
Changing the Table Prefix @@ -314,8 +329,8 @@ catch (JobRestartException e) { meta data tables is needed within the same schema, then the table prefix will need to be changed: - <job-repository id="jobRepository" - table-prefix="SYSTEM.TEST_" /> + table-prefix="SYSTEM.TEST_"]]> Given the above changes, every query to the meta data tables will be prefixed with "SYSTEM.TEST_". BATCH_JOB_EXECUTION will be referred to @@ -327,6 +342,8 @@ catch (JobRestartException e) {
+ +
In-Memory Repository @@ -337,10 +354,10 @@ catch (JobRestartException e) { this reason, Spring batch provides an in-memory Map version of the job repository: - <bean id="jobRepository" - class="org.springframework.batch.core.repository.support.MapJobRepositoryFactoryBean"> - <property name="transactionManager" ref="transactionManager"/> -</bean> + + +]]> Note that the in-memory repository is volatile and so does not allow restart between JVM instances. It also cannot guarantee that two @@ -356,6 +373,8 @@ catch (JobRestartException e) { ResourcelessTransactionManager useful.
+ +
Non-standard Database Types in a Repository @@ -366,10 +385,10 @@ catch (JobRestartException e) { shortcut and use it to set the database type to the closest match: - <bean id="jobRepository" class="org...JobRepositoryFactoryBean"> - <property name="databaseType" value="db2"/> - <property name="dataSource" ref="dataSource"/> -</bean> + + + +]]> (The JobRepositoryFactoryBean tries to auto-detect the database type from the DataSource @@ -384,6 +403,8 @@ catch (JobRestartException e) { interfaces that the SimpleJobRepository depends on and wire one up manually in the normal Spring way.
+ +
@@ -395,10 +416,10 @@ catch (JobRestartException e) { a JobRepository, in order to obtain an execution: - <bean id="jobLauncher" - class="org.springframework.batch.core.launch.support.SimpleJobLauncher"> - <property name="jobRepository" ref="jobRepository" /> -</bean> + + +]]> Once a JobExecution is @@ -442,13 +463,13 @@ catch (JobRestartException e) { configured to allow for this scenario by configuring a TaskExecutor: - <bean id="jobLauncher" - class="org.springframework.batch.core.launch.support.SimpleJobLauncher"> - <property name="jobRepository" ref="jobRepository" /> - <property name="taskExecutor"> - <bean class="org.springframework.core.task.SimpleAsyncTaskExecutor" /> - </property> -</bean> + + + + + +]]> Any implementation of the spring TaskExecutor interface can be used to control how jobs are asynchronously @@ -545,27 +566,26 @@ catch (JobRestartException e) { name second. All arguments after these are considered to be JobParameters and must be in the format of 'name=value': - bash$ java CommandLineJobRunner endOfDayJob.xml endOfDay schedule.date(date)=2007/05/05 + bash$ - In most cases you would want to use a manifest to - declare your main class in a jar, but for simplicity, the - class was used directly. This example is using the same - 'EndOfDay' example from the domain - section. The first argument is 'endOfDayJob.xml', which - is the Spring ApplicationContext - containing the + In most cases you would want to use a manifest to declare your + main class in a jar, but for simplicity, the class was used directly. + This example is using the same 'EndOfDay' example from the domain section. The first argument is + 'endOfDayJob.xml', which is the Spring + ApplicationContext containing the Job. The second argument, 'endOfDay' represents the job name. The final argument, 'schedule.date(date)=2007/05/05' will be converted into JobParameters. An example of the XML configuration is below: - <job id="endOfDay"> - <step id="step1" parent="simpleStep" /> -</job> + + + -<!-- Launcher details removed for clarity --> -<beans:bean id="jobLauncher" - class="org.springframework.batch.core.launch.support.SimpleJobLauncher" /> + +]]> This example is overly simplistic, since there are many more requirements to a run a batch job in Spring Batch in general, but it @@ -603,11 +623,11 @@ catch (JobRestartException e) { to a number using the ExitCodeMapper interface: - public interface ExitCodeMapper { + +}]]> The essential contract of an ExitCodeMapper is that, given a string exit @@ -666,7 +686,7 @@ catch (JobRestartException e) { is required when handling an HttpRequest. An example is below: - @Controller + +}]]>
@@ -736,9 +756,9 @@ public class JobLauncherController { query the repository for existing executions. This functionality is provided by the JobExplorer interface: - public interface JobExplorer { + getJobInstances(String jobName, int start, int count); JobExecution getJobExecution(Long executionId); @@ -746,10 +766,10 @@ public class JobLauncherController { JobInstance getJobInstance(Long instanceId); - List<JobExecution> getJobExecutions(JobInstance jobInstance); + List getJobExecutions(JobInstance jobInstance); - Set<JobExecution> findRunningJobExecutions(String jobName); -} + Set findRunningJobExecutions(String jobName); +}]]> As is evident from the method signatures above, JobExplorer is a read-only version of the @@ -757,8 +777,8 @@ public class JobLauncherController { JobRepository, it can be easily configured via a factory bean: - <bean id="jobExplorer" class="org.spr...JobExplorerFactoryBean" - p:dataSource-ref="dataSource" /> + ]]> Earlier in this chapter, it was mentioned that the table prefix of the @@ -767,8 +787,8 @@ public class JobLauncherController { JobExplorer is working with the same tables, it too needs the ability to set a prefix: - <bean id="jobExplorer" class="org.spr...JobExplorerFactoryBean" - p:dataSource-ref="dataSource" p:tablePrefix="BATCH_" /> + p:tablePrefix="BATCH_" ]]>
@@ -784,7 +804,7 @@ public class JobLauncherController { the framework and this is based on a simple map from job name to job instance. It is configured simply like this: - <bean id="jobRegistry" class="org.spr...MapJobRegistry" /> + ]]> There are two ways to populate a JobRegistry automatically: using a bean post processor and using a registrar lifecycle component. These @@ -796,9 +816,9 @@ public class JobLauncherController { This is a bean post-processor that can register all jobs as they are created: - <bean id="jobRegistryBeanPostProcessor" class="org.spr...JobRegistryBeanPostProcessor"> - <property name="jobRegistry" ref="jobRegistry"/> -</bean> + + +]]> Athough it is not strictly necessary the post-processor in the example has been given an id so that it can be included in child @@ -823,18 +843,18 @@ public class JobLauncherController { integrate jobs contributed from separate modules of an application. - <bean class="org.spr...AutomaticJobRegistrar"> - <property name="applicationContextFactories"> - <bean class="org.spr...ClasspathXmlApplicationContextsFactoryBean"> - <property name="resources" value="classpath*:/config/job*.xml" /> - </bean> - </property> - <property name="jobLoader"> - <bean class="org.spr...DefaultJobLoader"> - <property name="jobRegistry" ref="jobRegistry" /> - </bean> - </property> -</bean> + + + + + + + + + + + +]]> The registrar has two mandatory properties, one is an array of ApplicationContextFactory (here created from a @@ -875,14 +895,14 @@ public class JobLauncherController { provides for these types of operations via the JobOperator interface: - public interface JobOperator { + getExecutions(long instanceId) throws NoSuchJobInstanceException; - List<Long> getJobInstances(String jobName, int start, int count) + List getJobInstances(String jobName, int start, int count) throws NoSuchJobException; - Set<Long> getRunningExecutions(String jobName) throws NoSuchJobException; + Set getRunningExecutions(String jobName) throws NoSuchJobException; String getParameters(long executionId) throws NoSuchJobExecutionException; @@ -902,12 +922,12 @@ public class JobLauncherController { String getSummary(long executionId) throws NoSuchJobExecutionException; - Map<Long, String> getStepExecutionSummaries(long executionId) + Map getStepExecutionSummaries(long executionId) throws NoSuchJobExecutionException; - Set<String> getJobNames(); + Set getJobNames(); -} +}]]> The above operations represent methods from many different interfaces, such as JobLauncher, @@ -917,20 +937,20 @@ public class JobLauncherController { implementation of JobOperator, SimpleJobOperator, has many dependencies: - <bean id="jobOperator" class="org.spr...SimpleJobOperator"> - <property name="jobExplorer"> - <bean class="org.spr...JobExplorerFactoryBean"> - <property name="dataSource" ref="dataSource" /> - </bean> - </property> - <property name="jobRepository" ref="jobRepository" /> - <property name="jobRegistry" ref="jobRegistry" /> - <property name="jobLauncher" ref="jobLauncher" /> -</bean> - - If you set the table prefix on the job repository, don't - forget to set it on the job explorer as well. + + + + + + + + + +]]> + + If you set the table prefix on the job repository, don't forget to set it on the job explorer as well. +
@@ -955,11 +975,11 @@ public class JobLauncherController { Job to force the Job to a new instance: - public interface JobParametersIncrementer { + +}]]> The contract of JobParametersIncrementer is that, given a Job, as shown below: - public class SampleIncrementer implements JobParametersIncrementer { + +}]]> In this example, the value with a key of 'run.id' is used to discriminate between JobInstances. If the @@ -995,9 +1015,9 @@ public class JobLauncherController { be associated with Job via the 'incrementer' attribute in the namespace: - <job id="footballJob" incrementer="sampleIncrementer"> + incrementer="sampleIncrementer" ... -</job> +]]>
@@ -1007,8 +1027,8 @@ public class JobLauncherController { JobOperator is gracefully stopping a Job: - Set<Long> executions = jobOperator.getRunningExecutions("sampleJob"); -jobOperator.stop(executions.iterator().next()); + executions = jobOperator.getRunningExecutions("sampleJob"); +jobOperator.stop(executions.iterator().next()); ]]> The shutdown is not immediate, since there is no way to force immediate shutdown, especially if the execution is currently in @@ -1019,5 +1039,30 @@ jobOperator.stop(executions.iterator().next()); BatchStatus.STOPPED, save it, then do the same for the JobExecution before finishing.
+ +
+ Aborting a Job + + A job execution which is FAILED can be + restartsed (if the Job is restartable). A job execution whose status is + ABORTED will not be restarted by the framework. + The ABORTED status is also used in step + executions to mark them as skippable in a restarted job execution: if a + job is executing and encounters a step that has been marked + ABORTED in the previous failed job execution, it + will move on to the next step (as determined by the job flow definition + and the step execution exit status). + + If the process died ("kill -9" or server + failure) the job is, of course, not running, but the JobRepository has + no way of knowing because no-one told it before the process died. You + have to tell it manually that you know that the execution either failed + or should be considered aborted (change its status to + FAILED or ABORTED) - it's + a business decision and there is no way to automate it. Only change the + status to FAILED if it is not restartable, or if + you know the restart data is valid. There is a utility in Spring Batch + Admin JobService to abort a job execution. +
From 415b372d6091040484e184138dad6a5f27ad11eb Mon Sep 17 00:00:00 2001 From: Dave Syer Date: Fri, 26 Nov 2010 21:22:03 +0800 Subject: [PATCH 12/16] BATCH-1659: add test for file exists (moved from setUpOutputFile to createNewFile) --- .../item/file/transform/DefaultFieldSet.java | 51 ++++++++++++++----- .../item/file/transform/FieldSetFactory.java | 15 +++++- .../batch/item/util/FileUtils.java | 2 +- .../batch/item/util/FileUtilsTests.java | 2 +- 4 files changed, 54 insertions(+), 16 deletions(-) diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/DefaultFieldSet.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/DefaultFieldSet.java index 2f70c9c8f5..2bef134a84 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/DefaultFieldSet.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/DefaultFieldSet.java @@ -49,7 +49,9 @@ public class DefaultFieldSet implements FieldSet { } private NumberFormat numberFormat = NumberFormat.getInstance(Locale.US); + private String grouping = ","; + private String decimal = "."; /** @@ -60,15 +62,15 @@ public class DefaultFieldSet implements FieldSet { private List names; /** - * The {@link NumberFormat} to use for parsing numbers. If unset the - * US locale will be used ('.' as decimal place). + * The {@link NumberFormat} to use for parsing numbers. If unset the US + * locale will be used ('.' as decimal place). * @param numberFormat the {@link NumberFormat} to use for number parsing */ public final void setNumberFormat(NumberFormat numberFormat) { this.numberFormat = numberFormat; if (numberFormat instanceof DecimalFormat) { - grouping = ""+((DecimalFormat)numberFormat).getDecimalFormatSymbols().getGroupingSeparator(); - decimal = ""+((DecimalFormat)numberFormat).getDecimalFormatSymbols().getDecimalSeparator(); + grouping = "" + ((DecimalFormat) numberFormat).getDecimalFormatSymbols().getGroupingSeparator(); + decimal = "" + ((DecimalFormat) numberFormat).getDecimalFormatSymbols().getDecimalSeparator(); } } @@ -81,11 +83,24 @@ public void setDateFormat(DateFormat dateFormat) { this.dateFormat = dateFormat; } + /** + * Create a FieldSet with anonymous tokens. They can only be retrieved by + * column number. + * @param tokens the token values + * @see FieldSet#readString(int) + */ public DefaultFieldSet(String[] tokens) { this.tokens = tokens == null ? null : (String[]) tokens.clone(); setNumberFormat(NumberFormat.getInstance(Locale.US)); } + /** + * Create a FieldSet with named tokens. The token values can then be + * retrieved either by name or by column number. + * @param tokens the token values + * @param names the names of the tokens + * @see FieldSet#readString(String) + */ public DefaultFieldSet(String[] tokens, String[] names) { Assert.notNull(tokens); Assert.notNull(names); @@ -438,7 +453,7 @@ public BigDecimal readBigDecimal(String name) { */ public BigDecimal readBigDecimal(int index, BigDecimal defaultValue) { String candidate = readAndTrim(index); - + if (!StringUtils.hasText(candidate)) { return defaultValue; } @@ -484,13 +499,17 @@ public Date readDate(int index) { return parseDate(readAndTrim(index), dateFormat); } - /* (non-Javadoc) - * @see org.springframework.batch.item.file.transform.FieldSet#readDate(int, java.util.Date) + /* + * (non-Javadoc) + * + * @see org.springframework.batch.item.file.transform.FieldSet#readDate(int, + * java.util.Date) */ public Date readDate(int index, Date defaultValue) { try { return readDate(index); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { return defaultValue; } } @@ -511,13 +530,17 @@ public Date readDate(String name) { } } - /* (non-Javadoc) - * @see org.springframework.batch.item.file.transform.FieldSet#readDate(int, java.util.Date) + /* + * (non-Javadoc) + * + * @see org.springframework.batch.item.file.transform.FieldSet#readDate(int, + * java.util.Date) */ public Date readDate(String name, Date defaultValue) { try { return readDate(name); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { return defaultValue; } } @@ -543,7 +566,8 @@ public Date readDate(int index, String pattern) { public Date readDate(int index, String pattern, Date defaultValue) { try { return readDate(index, pattern); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { return defaultValue; } } @@ -573,7 +597,8 @@ public Date readDate(String name, String pattern) { public Date readDate(String name, String pattern, Date defaultValue) { try { return readDate(name, pattern); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { return defaultValue; } } diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/FieldSetFactory.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/FieldSetFactory.java index c547cff673..27f6c318ae 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/FieldSetFactory.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/file/transform/FieldSetFactory.java @@ -11,8 +11,21 @@ */ public interface FieldSetFactory { - FieldSet create(String[] names, String[] values); + /** + * Create a FieldSet with named tokens. The token values can then be + * retrieved either by name or by column number. + * @param values the token values + * @param names the names of the tokens + * @see DefaultFieldSet#readString(String) + */ + FieldSet create(String[] values, String[] names); + /** + * Create a FieldSet with anonymous tokens. They can only be retrieved by + * column number. + * @param values the token values + * @see FieldSet#readString(int) + */ FieldSet create(String[] values); } diff --git a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java index 5be6d29231..8b5df89701 100644 --- a/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java +++ b/spring-batch-infrastructure/src/main/java/org/springframework/batch/item/util/FileUtils.java @@ -95,7 +95,7 @@ public static boolean createNewFile(File file) throws IOException { } try { - return file.createNewFile(); + return file.createNewFile() && file.exists(); } catch (IOException e) { // On some filesystems you can get an exception here even though the diff --git a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/util/FileUtilsTests.java b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/util/FileUtilsTests.java index b22c30d787..0eaed6f111 100644 --- a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/util/FileUtilsTests.java +++ b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/util/FileUtilsTests.java @@ -141,7 +141,7 @@ public boolean exists() { }; try{ FileUtils.setUpOutputFile(file, false, false); - fail(); + fail("Expected IOException because file doesn't exist"); }catch(ItemStreamException ex){ String message = ex.getMessage(); assertTrue("Wrong message: "+message, message.startsWith("Output file was not created")); From d5e86f229df6abb11a572dcbb1571c5418058a25 Mon Sep 17 00:00:00 2001 From: Dave Syer Date: Tue, 7 Dec 2010 01:33:13 +0800 Subject: [PATCH 13/16] Fix Java 5 compile --- .../batch/item/file/FlatFileItemReaderTests.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java index 50f42c5d3e..62ea8e522a 100644 --- a/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java +++ b/spring-batch-infrastructure/src/test/java/org/springframework/batch/item/file/FlatFileItemReaderTests.java @@ -387,8 +387,6 @@ public void testStrictness() throws Exception { @Test public void testMappingExceptionWrapping() throws Exception { LineMapper exceptionLineMapper = new LineMapper() { - - @Override public String mapLine(String line, int lineNumber) throws Exception { if (lineNumber == 2) { throw new Exception("Couldn't map line 2"); From 65afd0b1203031355083ef4d71fa73e6260e524c Mon Sep 17 00:00:00 2001 From: Josh Long Date: Mon, 13 Dec 2010 03:39:26 -0800 Subject: [PATCH 14/16] adding support for inline parsing of steps for partition steps. BATCH-1659 --- .../xml/StepParserStepFactoryBean.java | 3 + .../xml/PartitionStepParserTests.java | 166 ++++++++++++------ .../batch/sandbox/Customer.java | 59 ------- .../batch/sandbox/CustomerRowMapper.java | 31 ---- .../batch/sandbox/LoggingItemWriter.java | 33 ---- .../batch/sandbox/PartionedStepClient.java | 45 ----- .../xml/PartitionStepParserTests-context.xml | 17 ++ 7 files changed, 129 insertions(+), 225 deletions(-) delete mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java delete mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java delete mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java delete mode 100644 spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java diff --git a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java index 23236ca87e..a8c9c674d2 100644 --- a/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java +++ b/spring-batch-core/src/main/java/org/springframework/batch/core/configuration/xml/StepParserStepFactoryBean.java @@ -74,6 +74,7 @@ * appropriate class for generating the {@link Step}. * * @author Dan Garrette + * @author Josh Long * @see SimpleStepFactoryBean * @see FaultTolerantStepFactoryBean * @see TaskletStep @@ -286,6 +287,8 @@ private void configurePartitionStep(PartitionStep ts) { handler = partitionHandler; } + + // BATCH-1659 if (handler instanceof TaskExecutorPartitionHandler) { try { TaskExecutorPartitionHandler taskExecutorPartitionHandler = (TaskExecutorPartitionHandler) handler; diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java index 2fe684da82..f48bfb4799 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -25,75 +26,126 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import org.springframework.batch.core.BatchStatus; -import org.springframework.batch.core.Job; -import org.springframework.batch.core.JobExecution; -import org.springframework.batch.core.JobParameters; -import org.springframework.batch.core.StepExecution; +import org.springframework.batch.core.*; +import org.springframework.batch.core.partition.support.PartitionStep; +import org.springframework.batch.core.partition.support.TaskExecutorPartitionHandler; import org.springframework.batch.core.repository.JobRepository; import org.springframework.batch.core.repository.support.MapJobRepositoryFactoryBean; +import org.springframework.batch.core.step.tasklet.TaskletStep; +import org.springframework.beans.BeansException; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; import org.springframework.test.context.ContextConfiguration; import org.springframework.test.context.junit4.SpringJUnit4ClassRunner; +import org.springframework.util.ReflectionUtils; /** * @author Dave Syer - * + * @author Josh Long */ @ContextConfiguration @RunWith(SpringJUnit4ClassRunner.class) -public class PartitionStepParserTests { - - @Autowired - @Qualifier("job1") - private Job job1; - - @Autowired - @Qualifier("job2") - private Job job2; - - @Autowired - private JobRepository jobRepository; - - @Autowired - private MapJobRepositoryFactoryBean mapJobRepositoryFactoryBean; - - @Before - public void setUp() { - mapJobRepositoryFactoryBean.clear(); - } - - @Test - public void testDefaultHandlerStep() throws Exception { - assertNotNull(job1); - JobExecution jobExecution = jobRepository.createJobExecution(job1.getName(), new JobParameters()); - job1.execute(jobExecution); - assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus()); - List stepNames = getStepNames(jobExecution); - assertEquals(3, stepNames.size()); - assertEquals("[s1, step1:partition0, step1:partition1]", stepNames.toString()); - } - - @Test - public void testHandlerRefStep() throws Exception { - assertNotNull(job2); - JobExecution jobExecution = jobRepository.createJobExecution(job2.getName(), new JobParameters()); - job2.execute(jobExecution); - assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus()); - List stepNames = getStepNames(jobExecution); - assertEquals(5, stepNames.size()); - assertEquals("[s2, s3, step1:partition0, step1:partition1, step1:partition2]", stepNames.toString()); - } - - private List getStepNames(JobExecution jobExecution) { - List list = new ArrayList(); - for (StepExecution stepExecution : jobExecution.getStepExecutions()) { - list.add(stepExecution.getStepName()); - } - Collections.sort(list); - return list; - } +public class PartitionStepParserTests implements ApplicationContextAware { + + @Autowired + @Qualifier("job1") + private Job job1; + + @Autowired + @Qualifier("job2") + private Job job2; + + @Autowired + @Qualifier("job3") + private Job job3; + + @Autowired + private JobRepository jobRepository; + + @Autowired + private MapJobRepositoryFactoryBean mapJobRepositoryFactoryBean; + + private ApplicationContext applicationContext; + + public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + @Before + public void setUp() { + mapJobRepositoryFactoryBean.clear(); + } + + @SuppressWarnings("unchecked") + private T accessPrivateField(Object o, String fieldName) { + Field field = ReflectionUtils.findField(o.getClass(), fieldName); + boolean previouslyAccessibleValue = field.isAccessible(); + field.setAccessible(true); + T val = (T) ReflectionUtils.getField(field, o); + field.setAccessible(previouslyAccessibleValue); + return val; + } + + /** + * BATCH-1659 we now support the ability define steps inline for partitioned steps. + * this demonstates that the execution proceeds as expected and that the partitionhandler has a reference to the inline step definition + */ + @Test + public void testNestedPartitionStepStepReference() throws Throwable { + assertNotNull("the reference to the job3 configured in the XML file must not be null", job3); + JobExecution jobExecution = jobRepository.createJobExecution(job3.getName(), new JobParameters()); + + job3.execute(jobExecution); + + for (StepExecution se : jobExecution.getStepExecutions()) { + String stepExecutionName = se.getStepName(); + if (stepExecutionName.equalsIgnoreCase("j3s1")) { // the partitioned step + PartitionStep partitionStep = (PartitionStep) this.applicationContext.getBean(stepExecutionName); + + // prove that the reference in the {@link TaskExecutorPartitionHandler} is the step configured inline + TaskExecutorPartitionHandler taskExecutorPartitionHandler = accessPrivateField(partitionStep, "partitionHandler"); + TaskletStep taskletStep = accessPrivateField(taskExecutorPartitionHandler, "step"); + + assertNotNull("the taskletStep wasn't configured with a step. " + + "We're trusting that the factory ensured " + + "a reference was given.", taskletStep); + } + } + assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus()); + } + + @Test + public void testDefaultHandlerStep() throws Exception { + assertNotNull(job1); + JobExecution jobExecution = jobRepository.createJobExecution(job1.getName(), new JobParameters()); + job1.execute(jobExecution); + assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus()); + List stepNames = getStepNames(jobExecution); + assertEquals(3, stepNames.size()); + assertEquals("[s1, step1:partition0, step1:partition1]", stepNames.toString()); + } + + @Test + public void testHandlerRefStep() throws Exception { + assertNotNull(job2); + JobExecution jobExecution = jobRepository.createJobExecution(job2.getName(), new JobParameters()); + job2.execute(jobExecution); + assertEquals(BatchStatus.COMPLETED, jobExecution.getStatus()); + List stepNames = getStepNames(jobExecution); + assertEquals(5, stepNames.size()); + assertEquals("[s2, s3, step1:partition0, step1:partition1, step1:partition2]", stepNames.toString()); + } + + private List getStepNames(JobExecution jobExecution) { + List list = new ArrayList(); + for (StepExecution stepExecution : jobExecution.getStepExecutions()) { + list.add(stepExecution.getStepName()); + } + Collections.sort(list); + return list; + } } diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java deleted file mode 100644 index 0e2d58c4ea..0000000000 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/Customer.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright 2006-2007 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.springframework.batch.sandbox; - - -/** - * @author Josh Long - */ -public class Customer { - - private long id; - - private String firstName; - - private String lastName; - - private String email; - - public Customer(long id, String firstName, String lastName, String email) { - this.id = id; - this.firstName = firstName; - this.lastName = lastName; - this.email = email; - } - - public String getEmail() { - return email; - } - - public long getId() { - return id; - } - - public String getFirstName() { - return firstName; - } - - public String getLastName() { - return lastName; - } - - @Override - public String toString() { - return "Customer{" + "id=" + id + ", firstName='" + firstName + '\'' + ", lastName='" + lastName + '\'' + ", email='" + email + '\'' + '}'; - } -} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java deleted file mode 100644 index 877b3ed69d..0000000000 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/CustomerRowMapper.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright 2006-2007 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.springframework.batch.sandbox; - -import org.springframework.jdbc.core.RowMapper; - -import java.sql.ResultSet; -import java.sql.SQLException; - - -/** - * @author Josh Long - */ -public class CustomerRowMapper implements RowMapper { - public Object mapRow(ResultSet resultSet, int i) throws SQLException { - return new Customer( resultSet.getLong( "ID") , resultSet.getString("FIRST_NAME") , resultSet.getString( "LAST_NAME") , resultSet.getString( "EMAIL")) ; - } -} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java deleted file mode 100644 index dce7224071..0000000000 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/LoggingItemWriter.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright 2006-2007 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.springframework.batch.sandbox; - -import org.springframework.batch.item.ItemWriter; - -import java.util.List; - - -public class LoggingItemWriter implements ItemWriter { - public void write(List items) throws Exception { - - System.out.println(); - - for( Customer c : items) - System.out.println( "customer = " + c.toString()); - - System.out.println( "----------------------------------------------------------------------------------------" ); - } -} diff --git a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java b/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java deleted file mode 100644 index 02ee335ecf..0000000000 --- a/spring-batch-core/src/test/java/org/springframework/batch/sandbox/PartionedStepClient.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright 2006-2007 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.springframework.batch.sandbox; - -import org.springframework.batch.core.Job; -import org.springframework.batch.core.JobExecution; -import org.springframework.batch.core.JobParameters; -import org.springframework.batch.core.JobParametersBuilder; -import org.springframework.batch.core.launch.JobLauncher; -import org.springframework.context.support.ClassPathXmlApplicationContext; - -import java.util.UUID; - -/** - * @author Josh Long - * @see {@link org.springframework.batch.core.partition.support.PartitionStep} - */ -public class PartionedStepClient { - public static void main (String [] args ) throws Throwable { - - - ClassPathXmlApplicationContext cax = new ClassPathXmlApplicationContext ("org/springframework/batch/sandbox/ps1.xml"); - - JobLauncher jobLauncher = (JobLauncher) cax.getBean("jobLauncher"); - Job job = (Job) cax.getBean("partitionedJob") ; - - - JobParameters parms = new JobParametersBuilder().addString("uid", UUID.randomUUID().toString()).toJobParameters(); - JobExecution execution = jobLauncher.run( job, parms); - - } -} diff --git a/spring-batch-core/src/test/resources/org/springframework/batch/core/configuration/xml/PartitionStepParserTests-context.xml b/spring-batch-core/src/test/resources/org/springframework/batch/core/configuration/xml/PartitionStepParserTests-context.xml index 114514d5a7..7a931a3931 100644 --- a/spring-batch-core/src/test/resources/org/springframework/batch/core/configuration/xml/PartitionStepParserTests-context.xml +++ b/spring-batch-core/src/test/resources/org/springframework/batch/core/configuration/xml/PartitionStepParserTests-context.xml @@ -21,6 +21,15 @@ + + + + + + + + + @@ -34,4 +43,12 @@ + + + + + + \ No newline at end of file From da6607579e4d3fec17ece046a45f115eadd00dc6 Mon Sep 17 00:00:00 2001 From: Josh Long Date: Sat, 18 Dec 2010 19:11:57 -0800 Subject: [PATCH 15/16] BATCH-1509 - removed the last vestige of the old sandbox directory.. --- .../org/springframework/batch/sandbox/ps1.xml | 75 ------------------- 1 file changed, 75 deletions(-) delete mode 100644 spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml diff --git a/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml b/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml deleted file mode 100644 index 32d5647d92..0000000000 --- a/spring-batch-core/src/test/resources/org/springframework/batch/sandbox/ps1.xml +++ /dev/null @@ -1,75 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - From 4a604aab9a5bcc04bc885b0131307d3df1bfda0f Mon Sep 17 00:00:00 2001 From: Josh Long Date: Sat, 18 Dec 2010 19:36:20 -0800 Subject: [PATCH 16/16] fixed a comment... --- .../batch/core/configuration/xml/PartitionStepParserTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java b/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java index f48bfb4799..7470cf1763 100644 --- a/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java +++ b/spring-batch-core/src/test/java/org/springframework/batch/core/configuration/xml/PartitionStepParserTests.java @@ -90,7 +90,7 @@ private T accessPrivateField(Object o, String fieldName) { } /** - * BATCH-1659 we now support the ability define steps inline for partitioned steps. + * BATCH-1509 we now support the ability define steps inline for partitioned steps. * this demonstates that the execution proceeds as expected and that the partitionhandler has a reference to the inline step definition */ @Test