You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2010/07/28 11:26:37 UTC

svn commit: r980003 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/component/file/strategy/ camel-core/src/test/java/org/apache/camel/component/file/ components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/

Author: davsclaus
Date: Wed Jul 28 09:26:36 2010
New Revision: 980003

URL: http://svn.apache.org/viewvc?rev=980003&view=rev
Log:
CAMEL-3003: file component now support using preMove with noop or delete option together.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveDeleteTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveNoopTest.java
      - copied, changed from r979974, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveDeleteTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveNoopTest.java
      - copied, changed from r979974, camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFileExpressionTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/FileProcessStrategyFactory.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDeleteProcessStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/FileProcessStrategyFactory.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/FileProcessStrategyFactory.java?rev=980003&r1=980002&r2=980003&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/FileProcessStrategyFactory.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/FileProcessStrategyFactory.java Wed Jul 28 09:26:36 2010
@@ -41,34 +41,40 @@ public final class FileProcessStrategyFa
         boolean isDelete = params.get("delete") != null;
         boolean isMove = moveExpression != null || preMoveExpression != null || moveFailedExpression != null;
 
-        if (isNoop) {
-            GenericFileNoOpProcessStrategy<File> strategy = new GenericFileNoOpProcessStrategy<File>();
-            strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
-            return strategy;
-        } else if (isDelete) {
+        if (isDelete) {
             GenericFileDeleteProcessStrategy<File> strategy = new GenericFileDeleteProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
+            if (preMoveExpression != null) {
+                GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
+                renamer.setExpression(preMoveExpression);
+                strategy.setBeginRenamer(renamer);
+            }
             if (moveFailedExpression != null) {
                 GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
                 renamer.setExpression(moveFailedExpression);
                 strategy.setFailureRenamer(renamer);
             }
             return strategy;
-        } else if (isMove) {
+        } else if (isMove || isNoop) {
             GenericFileRenameProcessStrategy<File> strategy = new GenericFileRenameProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
-            if (moveExpression != null) {
-                GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
-                renamer.setExpression(moveExpression);
-                strategy.setCommitRenamer(renamer);
-            } else {
-                strategy.setCommitRenamer(getDefaultCommitRenamer(context));
+            if (!isNoop) {
+                // move on commit is only possible if not noop
+                if (moveExpression != null) {
+                    GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
+                    renamer.setExpression(moveExpression);
+                    strategy.setCommitRenamer(renamer);
+                } else {
+                    strategy.setCommitRenamer(getDefaultCommitRenamer(context));
+                }
             }
+            // both move and noop supports pre move
             if (preMoveExpression != null) {
                 GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
                 renamer.setExpression(preMoveExpression);
                 strategy.setBeginRenamer(renamer);
             }
+            // both move and noop supports move failed
             if (moveFailedExpression != null) {
                 GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
                 renamer.setExpression(moveFailedExpression);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDeleteProcessStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDeleteProcessStrategy.java?rev=980003&r1=980002&r2=980003&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDeleteProcessStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDeleteProcessStrategy.java Wed Jul 28 09:26:36 2010
@@ -25,7 +25,27 @@ import org.apache.camel.component.file.G
 public class GenericFileDeleteProcessStrategy<T> extends GenericFileProcessStrategySupport<T> {
 
     private GenericFileRenamer<T> failureRenamer;
-    
+    private GenericFileRenamer<T> beginRenamer;
+
+    @Override
+    public boolean begin(GenericFileOperations<T> operations, GenericFileEndpoint<T> endpoint, Exchange exchange, GenericFile<T> file) throws Exception {
+        // must invoke super
+        boolean result = super.begin(operations, endpoint, exchange, file);
+        if (!result) {
+            return false;
+        }
+
+        if (beginRenamer != null) {
+            GenericFile<T> newName = beginRenamer.renameFile(exchange, file);
+            GenericFile<T> to = renameFile(operations, file, newName);
+            if (to != null) {
+                to.bindToExchange(exchange);
+            }
+        }
+
+        return true;
+    }
+
     @Override
     public void commit(GenericFileOperations<T> operations, GenericFileEndpoint<T> endpoint, Exchange exchange, GenericFile<T> file) throws Exception {
         // must invoke super
@@ -82,5 +102,11 @@ public class GenericFileDeleteProcessStr
         this.failureRenamer = failureRenamer;
     }
 
+    public GenericFileRenamer<T> getBeginRenamer() {
+        return beginRenamer;
+    }
 
+    public void setBeginRenamer(GenericFileRenamer<T> beginRenamer) {
+        this.beginRenamer = beginRenamer;
+    }
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java?rev=980003&r1=980002&r2=980003&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java Wed Jul 28 09:26:36 2010
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.component.file.strategy;
 
+import java.io.File;
 import java.util.Map;
 
 import org.apache.camel.CamelContext;
@@ -40,27 +41,36 @@ public final class GenericFileProcessStr
         boolean isDelete = params.get("delete") != null;
         boolean isMove = moveExpression != null || preMoveExpression != null || moveFailedExpression != null;
 
-        if (isNoop) {
-            GenericFileNoOpProcessStrategy<T> strategy = new GenericFileNoOpProcessStrategy<T>();
-            strategy.setExclusiveReadLockStrategy((GenericFileExclusiveReadLockStrategy<T>) getExclusiveReadLockStrategy(params));
-            return strategy;
-        } else if (isDelete) {
+        if (isDelete) {
             GenericFileDeleteProcessStrategy<T> strategy = new GenericFileDeleteProcessStrategy<T>();
             strategy.setExclusiveReadLockStrategy((GenericFileExclusiveReadLockStrategy<T>) getExclusiveReadLockStrategy(params));
+            if (preMoveExpression != null) {
+                GenericFileExpressionRenamer<T> renamer = new GenericFileExpressionRenamer<T>();
+                renamer.setExpression(preMoveExpression);
+                strategy.setBeginRenamer(renamer);
+            }
+            if (moveFailedExpression != null) {
+                GenericFileExpressionRenamer<T> renamer = new GenericFileExpressionRenamer<T>();
+                renamer.setExpression(moveFailedExpression);
+                strategy.setFailureRenamer(renamer);
+            }
             return strategy;
-        } else if (isMove) {
+        } else if (isMove || isNoop) {
             GenericFileRenameProcessStrategy<T> strategy = new GenericFileRenameProcessStrategy<T>();
             strategy.setExclusiveReadLockStrategy((GenericFileExclusiveReadLockStrategy<T>) getExclusiveReadLockStrategy(params));
-            if (moveExpression != null) {
+            if (!isNoop && moveExpression != null) {
+                // move on commit is only possible if not noop
                 GenericFileExpressionRenamer<T> renamer = new GenericFileExpressionRenamer<T>();
                 renamer.setExpression(moveExpression);
                 strategy.setCommitRenamer(renamer);
             }
+            // both move and noop supports pre move
             if (moveFailedExpression != null) {
                 GenericFileExpressionRenamer<T> renamer = new GenericFileExpressionRenamer<T>();
                 renamer.setExpression(moveFailedExpression);
                 strategy.setFailureRenamer(renamer);
             }
+            // both move and noop supports pre move
             if (preMoveExpression != null) {
                 GenericFileExpressionRenamer<T> renamer = new GenericFileExpressionRenamer<T>();
                 renamer.setExpression(preMoveExpression);

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveDeleteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveDeleteTest.java?rev=980003&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveDeleteTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveDeleteTest.java Wed Jul 28 09:26:36 2010
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.file;
+
+import java.io.File;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision: 895965 $
+ */
+public class FileConsumerPreMoveDeleteTest extends ContextTestSupport {
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/premove");
+        super.setUp();
+    }
+
+    public void testPreMoveDelete() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+
+        template.sendBodyAndHeader("file://target/premove", "Hello World", Exchange.FILE_NAME, "hello.txt");
+
+        assertMockEndpointsSatisfied();
+
+        // and file should still be deleted
+        Thread.sleep(1000);
+
+        File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
+        assertFalse("Pre move file should have been deleted", pre.exists());
+    }
+
+    public void testPreMoveDeleteSameFileTwice() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceivedInAnyOrder("Hello World", "Hello Again World");
+
+        template.sendBodyAndHeader("file://target/premove", "Hello World", Exchange.FILE_NAME, "hello.txt");
+        // give time for consumer to process this file before we drop the next file
+        Thread.sleep(2000);
+        template.sendBodyAndHeader("file://target/premove", "Hello Again World", Exchange.FILE_NAME, "hello.txt");
+        // give time for consumer to process this file before we drop the next file
+
+        assertMockEndpointsSatisfied();
+
+        // and file should still be deleted
+        Thread.sleep(1000);
+
+        File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
+        assertFalse("Pre move file should have been deleted", pre.exists());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("file://target/premove?preMove=work&delete=true&idempotent=false")
+                    .process(new MyPreMoveCheckerProcessor())
+                    .to("mock:result");
+            }
+        };
+    }
+
+    public static class MyPreMoveCheckerProcessor implements Processor {
+
+        public void process(Exchange exchange) throws Exception {
+            File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
+            assertTrue("Pre move file should exist", pre.exists());
+        }
+    }
+}

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveNoopTest.java (from r979974, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveNoopTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveNoopTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveTest.java&r1=979974&r2=980003&rev=980003&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerPreMoveNoopTest.java Wed Jul 28 09:26:36 2010
@@ -27,7 +27,7 @@ import org.apache.camel.component.mock.M
 /**
  * @version $Revision$
  */
-public class FileConsumerPreMoveTest extends ContextTestSupport {
+public class FileConsumerPreMoveNoopTest extends ContextTestSupport {
 
     @Override
     protected void setUp() throws Exception {
@@ -35,16 +35,22 @@ public class FileConsumerPreMoveTest ext
         super.setUp();
     }
 
-    public void testPreMove() throws Exception {
+    public void testPreMoveNoop() throws Exception {
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedMessageCount(1);
 
         template.sendBodyAndHeader("file://target/premove", "Hello World", Exchange.FILE_NAME, "hello.txt");
 
         assertMockEndpointsSatisfied();
+
+        // and file should still be there in premove directory
+        Thread.sleep(1000);
+
+        File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
+        assertTrue("Pre move file should exist", pre.exists());
     }
 
-    public void testPreMoveSameFileTwice() throws Exception {
+    public void testPreMoveNoopSameFileTwice() throws Exception {
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedBodiesReceivedInAnyOrder("Hello World", "Hello Again World");
 
@@ -53,7 +59,14 @@ public class FileConsumerPreMoveTest ext
         Thread.sleep(2000);
         template.sendBodyAndHeader("file://target/premove", "Hello Again World", Exchange.FILE_NAME, "hello.txt");
         // give time for consumer to process this file before we drop the next file
+
         assertMockEndpointsSatisfied();
+
+        // and file should still be there in premove directory
+        Thread.sleep(1000);
+
+        File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
+        assertTrue("Pre move file should exist", pre.exists());
     }
 
     @Override
@@ -61,7 +74,7 @@ public class FileConsumerPreMoveTest ext
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                from("file://target/premove?preMove=work/work-${file:name}")
+                from("file://target/premove?preMove=work&noop=true&idempotent=false")
                     .process(new MyPreMoveCheckerProcessor())
                     .to("mock:result");
             }
@@ -71,7 +84,7 @@ public class FileConsumerPreMoveTest ext
     public static class MyPreMoveCheckerProcessor implements Processor {
 
         public void process(Exchange exchange) throws Exception {
-            File pre = new File("target/premove/work/work-hello.txt").getAbsoluteFile();
+            File pre = new File("target/premove/work/hello.txt").getAbsoluteFile();
             assertTrue("Pre move file should exist", pre.exists());
         }
     }

Added: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveDeleteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveDeleteTest.java?rev=980003&view=auto
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveDeleteTest.java (added)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveDeleteTest.java Wed Jul 28 09:26:36 2010
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.file.remote;
+
+import java.io.File;
+
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.Producer;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit test to test preMove with delete option.
+ */
+public class FromFtpPreMoveDeleteTest extends FtpServerTestSupport {
+
+    private String getFtpUrl() {
+        return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&preMove=work&delete=true";
+    }
+
+    @Override
+    @Before
+    public void setUp() throws Exception {
+        super.setUp();
+        prepareFtpServer();
+    }
+
+    @Test
+    public void testPreMoveDelete() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedBodiesReceived("Hello World this file will be moved");
+
+        mock.assertIsSatisfied();
+
+        // and file should be deleted
+        Thread.sleep(1000);
+        File file = new File(FTP_ROOT_DIR + "movefile/work/hello.txt");
+        file = file.getAbsoluteFile();
+        assertFalse("The file should have been deleted", file.exists());
+    }
+
+    private void prepareFtpServer() throws Exception {
+        // prepares the FTP Server by creating a file on the server that we want to unit
+        // test that we can pool and store as a local file
+        Endpoint endpoint = context.getEndpoint(getFtpUrl());
+        Exchange exchange = endpoint.createExchange();
+        exchange.getIn().setBody("Hello World this file will be moved");
+        exchange.getIn().setHeader(Exchange.FILE_NAME, "hello.txt");
+        Producer producer = endpoint.createProducer();
+        producer.start();
+        producer.process(exchange);
+        producer.stop();
+    }
+    
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            public void configure() throws Exception {
+                from(getFtpUrl()).process(new Processor() {
+                    public void process(Exchange exchange) throws Exception {
+                        // assert the file is pre moved
+                        File file = new File(FTP_ROOT_DIR + "movefile/work/hello.txt");
+                        file = file.getAbsoluteFile();
+                        assertTrue("The file should have been moved", file.exists());
+                    }
+                }).to("mock:result");
+            }
+        };
+    }
+}
\ No newline at end of file

Copied: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveNoopTest.java (from r979974, camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFileExpressionTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveNoopTest.java?p2=camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveNoopTest.java&p1=camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFileExpressionTest.java&r1=979974&r2=980003&rev=980003&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFileExpressionTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveNoopTest.java Wed Jul 28 09:26:36 2010
@@ -28,13 +28,12 @@ import org.junit.Before;
 import org.junit.Test;
 
 /**
- * Unit test to test preMove option.
+ * Unit test to test preMove with noop option.
  */
-public class FromFtpPreMoveFileExpressionTest extends FtpServerTestSupport {
+public class FromFtpPreMoveNoopTest extends FtpServerTestSupport {
 
     private String getFtpUrl() {
-        return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false&consumer.delay=5000"
-                + "&preMove=../inprogress/${file:name.noext}.bak";
+        return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&preMove=work&noop=true";
     }
 
     @Override
@@ -45,12 +44,18 @@ public class FromFtpPreMoveFileExpressio
     }
 
     @Test
-    public void testPollFileAndShouldBeMoved() throws Exception {
+    public void testPreMoveNoop() throws Exception {
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedMessageCount(1);
         mock.expectedBodiesReceived("Hello World this file will be moved");
 
         mock.assertIsSatisfied();
+
+        // and file should be kept there
+        Thread.sleep(1000);
+        File file = new File(FTP_ROOT_DIR + "movefile/work/hello.txt");
+        file = file.getAbsoluteFile();
+        assertTrue("The file should exists", file.exists());
     }
 
     private void prepareFtpServer() throws Exception {
@@ -64,11 +69,6 @@ public class FromFtpPreMoveFileExpressio
         producer.start();
         producer.process(exchange);
         producer.stop();
-
-        // assert file is created
-        File file = new File(FTP_ROOT_DIR + "movefile/hello.txt");
-        file = file.getAbsoluteFile();
-        assertTrue("The file should exists", file.exists());
     }
     
     protected RouteBuilder createRouteBuilder() throws Exception {
@@ -77,7 +77,7 @@ public class FromFtpPreMoveFileExpressio
                 from(getFtpUrl()).process(new Processor() {
                     public void process(Exchange exchange) throws Exception {
                         // assert the file is pre moved
-                        File file = new File(FTP_ROOT_DIR + "inprogress/hello.bak");
+                        File file = new File(FTP_ROOT_DIR + "movefile/work/hello.txt");
                         file = file.getAbsoluteFile();
                         assertTrue("The file should have been moved", file.exists());
                     }