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 2009/02/08 14:13:15 UTC

svn commit: r742087 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/component/file/ 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/main/...

Author: davsclaus
Date: Sun Feb  8 13:13:14 2009
New Revision: 742087

URL: http://svn.apache.org/viewvc?rev=742087&view=rev
Log:
CAMEL-1324: Consolidated move and pre move options for file/ftp component.

Removed:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileDefaultRenamer.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/NewFileEndpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileExpressionRenamer.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategyFactory.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameExclusiveReadLockStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileLockExclusiveReadLockStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileProcessStrategyFactory.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewMarkerFileExclusiveReadLockStrategy.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginAndCommitRenameStrategyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginRenameStrategyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerCommitRenameStrategyTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileRouteTest.java
    camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileConsumer.java
    camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java
    camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileProducer.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePostfixTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePrefixTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFileTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePostfixTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePrefixTest.java
    camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FtpIllegalOptionsTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java Sun Feb  8 13:13:14 2009
@@ -58,10 +58,6 @@
     protected boolean recursive;
     protected boolean delete;
     protected String tempPrefix;
-    protected String moveNamePrefix;
-    protected String moveNamePostfix;
-    protected String preMoveNamePrefix;
-    protected String preMoveNamePostfix;
     protected String excludeNamePrefix;
     protected String excludeNamePostfix;
     protected String includeNamePrefix;
@@ -159,38 +155,6 @@
         this.noop = noop;
     }
 
-    public String getMoveNamePrefix() {
-        return moveNamePrefix;
-    }
-
-    public void setMoveNamePrefix(String moveNamePrefix) {
-        this.moveNamePrefix = moveNamePrefix;
-    }
-
-    public String getMoveNamePostfix() {
-        return moveNamePostfix;
-    }
-
-    public void setMoveNamePostfix(String moveNamePostfix) {
-        this.moveNamePostfix = moveNamePostfix;
-    }
-
-    public String getPreMoveNamePrefix() {
-        return preMoveNamePrefix;
-    }
-
-    public void setPreMoveNamePrefix(String preMoveNamePrefix) {
-        this.preMoveNamePrefix = preMoveNamePrefix;
-    }
-
-    public String getPreMoveNamePostfix() {
-        return preMoveNamePostfix;
-    }
-
-    public void setPreMoveNamePostfix(String preMoveNamePostfix) {
-        this.preMoveNamePostfix = preMoveNamePostfix;
-    }
-
     public String getExcludeNamePrefix() {
         return excludeNamePrefix;
     }
@@ -439,19 +403,8 @@
     }
 
     /**
-     * Should the file be moved after consuming?
-     *
-     * @deprecated not used
-     */
-    public boolean isMoveFile() {
-        return moveNamePostfix != null || moveNamePrefix != null
-                || preMoveNamePostfix != null || preMoveNamePrefix != null
-                || moveExpression != null;
-    }
-
-    /**
      * Configures the given message with the file which sets the body to the
-     * file object and sets the {@link FileComponent#HEADER_FILE_NAME} header.
+     * file object.
      */
     public void configureMessage(GenericFile<T> file, Message message) {
         message.setBody(file);
@@ -484,18 +437,6 @@
         if (isDelete()) {
             params.put("delete", Boolean.toString(true));
         }
-        if (moveNamePrefix != null) {
-            params.put("moveNamePrefix", moveNamePrefix);
-        }
-        if (moveNamePostfix != null) {
-            params.put("moveNamePostfix", moveNamePostfix);
-        }
-        if (preMoveNamePrefix != null) {
-            params.put("preMoveNamePrefix", preMoveNamePrefix);
-        }
-        if (preMoveNamePostfix != null) {
-            params.put("preMoveNamePostfix", preMoveNamePostfix);
-        }
         if (moveExpression != null) {
             params.put("moveExpression", moveExpression);
         }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/NewFileEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/NewFileEndpoint.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/NewFileEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/NewFileEndpoint.java Sun Feb  8 13:13:14 2009
@@ -48,8 +48,8 @@
 
         NewFileConsumer result = new NewFileConsumer(this, processor, operations);
 
-        if (isDelete() && (getMoveNamePrefix() != null || getMoveNamePostfix() != null || getMoveExpression() != null)) {
-            throw new IllegalArgumentException("You cannot set both delete=true and either a (moveNamePrefix, moveNamePostfix or moveExpression) option");
+        if (isDelete() && getMoveExpression() != null) {
+            throw new IllegalArgumentException("You cannot set both delete=true and moveExpression options");
         }
 
         // if noop=true then idempotent should also be configured

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileExpressionRenamer.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileExpressionRenamer.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileExpressionRenamer.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileExpressionRenamer.java Sun Feb  8 13:13:14 2009
@@ -24,6 +24,13 @@
 public class GenericFileExpressionRenamer<T> implements GenericFileRenamer<T> {
     private Expression expression;
 
+    public GenericFileExpressionRenamer() {
+    }
+
+    public GenericFileExpressionRenamer(Expression expression) {
+        this.expression = expression;
+    }
+
     public GenericFile<T> renameFile(GenericFileExchange<T> exchange, GenericFile<T> file) {
         ObjectHelper.notNull(expression, "expression");
 

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=742087&r1=742086&r2=742087&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 Sun Feb  8 13:13:14 2009
@@ -28,19 +28,14 @@
     private GenericFileProcessStrategyFactory() {
     }
 
+    @SuppressWarnings("unchecked")
     public static GenericFileProcessStrategy createGenericFileProcessStrategy(Map<String, Object> params) {
 
         // We assume a value is present only if its value not null for String and 'true' for boolean
         boolean isNoop = params.get("noop") != null;
         boolean isDelete = params.get("delete") != null;
-        String moveNamePrefix = (String) params.get("moveNamePrefix");
-        String moveNamePostfix = (String) params.get("moveNamePostfix");
-        String preMoveNamePrefix = (String) params.get("preMoveNamePrefix");
-        String preMoveNamePostfix = (String) params.get("preMoveNamePostfix");
         Expression moveExpression = (Expression) params.get("moveExpression");
         Expression preMoveExpression = (Expression) params.get("preMoveExpression");
-        boolean move = moveNamePrefix != null || moveNamePostfix != null;
-        boolean preMove = preMoveNamePrefix != null || preMoveNamePostfix != null;
 
         if (isNoop) {
             GenericFileNoOpProcessStrategy strategy = new GenericFileNoOpProcessStrategy();
@@ -50,16 +45,6 @@
             GenericFileDeleteProcessStrategy strategy = new GenericFileDeleteProcessStrategy();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
             return strategy;
-        } else if (move || preMove) {
-            GenericFileRenameProcessStrategy strategy = new GenericFileRenameProcessStrategy();
-            strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
-            if (move) {
-                strategy.setCommitRenamer(new GenericFileDefaultRenamer(moveNamePrefix, moveNamePostfix));
-            }
-            if (preMove) {
-                strategy.setBeginRenamer(new GenericFileDefaultRenamer(preMoveNamePrefix, preMoveNamePostfix));
-            }
-            return strategy;
         } else if (moveExpression != null || preMoveExpression != null) {
             GenericFileRenameProcessStrategy strategy = new GenericFileRenameProcessStrategy();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameExclusiveReadLockStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameExclusiveReadLockStrategy.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameExclusiveReadLockStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameExclusiveReadLockStrategy.java Sun Feb  8 13:13:14 2009
@@ -54,7 +54,7 @@
             if (timeout > 0) {
                 long delta = System.currentTimeMillis() - start;
                 if (delta > timeout) {
-                    LOG.debug("Could not acquire read lock within " + timeout + " millis. Will skip the file: " + file);
+                    LOG.debug("Cannot acquire read lock within " + timeout + " millis. Will skip the file: " + file);
                     // we could not get the lock within the timeout period, so return false
                     return false;
                 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java Sun Feb  8 13:13:14 2009
@@ -34,19 +34,6 @@
     public GenericFileRenameProcessStrategy() {
     }
 
-    public GenericFileRenameProcessStrategy(String namePrefix, String namePostfix) {
-        this(new GenericFileDefaultRenamer<T>(namePrefix, namePostfix), null);
-    }
-
-    public GenericFileRenameProcessStrategy(String namePrefix, String namePostfix, String preNamePrefix, String preNamePostfix) {
-        this(new GenericFileDefaultRenamer<T>(namePrefix, namePostfix), new GenericFileDefaultRenamer<T>(preNamePrefix, preNamePostfix));
-    }
-
-    public GenericFileRenameProcessStrategy(GenericFileRenamer<T> commitRenamer, GenericFileRenamer<T> beginRenamer) {
-        this.commitRenamer = commitRenamer;
-        this.beginRenamer = beginRenamer;
-    }
-
     @Override
     public boolean begin(GenericFileOperations<T> operations, GenericFileEndpoint<T> endpoint, GenericFileExchange<T> exchange, GenericFile<T> file) throws Exception {
         // must invoke super

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileLockExclusiveReadLockStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileLockExclusiveReadLockStrategy.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileLockExclusiveReadLockStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileLockExclusiveReadLockStrategy.java Sun Feb  8 13:13:14 2009
@@ -60,7 +60,7 @@
                 if (timeout > 0) {
                     long delta = System.currentTimeMillis() - start;
                     if (delta > timeout) {
-                        LOG.debug("Could not acquire read lock within " + timeout + " millis. Will skip the file: " + target);
+                        LOG.debug("Cannot acquire read lock within " + timeout + " millis. Will skip the file: " + target);
                         // we could not get the lock within the timeout period, so return false
                         return false;
                     }
@@ -79,8 +79,8 @@
                     }
 
                     // store lock so we can release it later
-                    exchange.setProperty("org.apache.camel.file.lock", lock);
-                    exchange.setProperty("org.apache.camel.file.lock.fileName", target.getName());
+                    exchange.setProperty("CamelFileLock", lock);
+                    exchange.setProperty("CamelFileLockName", target.getName());
 
                     exclusive = true;
                 } else {
@@ -105,8 +105,8 @@
 
     public void releaseExclusiveReadLock(GenericFileOperations<File> fileGenericFileOperations,
                                          GenericFile<File> fileGenericFile, Exchange exchange) throws Exception {
-        FileLock lock = ExchangeHelper.getMandatoryProperty(exchange, "org.apache.camel.file.lock", FileLock.class);
-        String lockFileName = ExchangeHelper.getMandatoryProperty(exchange, "org.apache.camel.file.lock.filename", String.class);
+        FileLock lock = ExchangeHelper.getMandatoryProperty(exchange, "CamelFileLock", FileLock.class);
+        String lockFileName = ExchangeHelper.getMandatoryProperty(exchange, "CamelFileLockName", String.class);
         Channel channel = lock.channel();
         try {
             lock.release();

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileProcessStrategyFactory.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileProcessStrategyFactory.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileProcessStrategyFactory.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewFileProcessStrategyFactory.java Sun Feb  8 13:13:14 2009
@@ -22,6 +22,7 @@
 import org.apache.camel.Expression;
 import org.apache.camel.component.file.GenericFileExclusiveReadLockStrategy;
 import org.apache.camel.component.file.GenericFileProcessStrategy;
+import org.apache.camel.language.simple.FileLanguage;
 import org.apache.camel.util.ObjectHelper;
 
 public final class NewFileProcessStrategyFactory {
@@ -34,14 +35,8 @@
         // We assume a value is present only if its value not null for String and 'true' for boolean
         boolean isNoop = params.get("noop") != null;
         boolean isDelete = params.get("delete") != null;
-        String moveNamePrefix = (String) params.get("moveNamePrefix");
-        String moveNamePostfix = (String) params.get("moveNamePostfix");
-        String preMoveNamePrefix = (String) params.get("preMoveNamePrefix");
-        String preMoveNamePostfix = (String) params.get("preMoveNamePostfix");
         Expression moveExpression = (Expression) params.get("moveExpression");
         Expression preMoveExpression = (Expression) params.get("preMoveExpression");
-        boolean move = moveNamePrefix != null || moveNamePostfix != null;
-        boolean preMove = preMoveNamePrefix != null || preMoveNamePostfix != null;
 
         if (isNoop) {
             GenericFileNoOpProcessStrategy<File> strategy = new GenericFileNoOpProcessStrategy<File>();
@@ -51,16 +46,6 @@
             GenericFileDeleteProcessStrategy<File> strategy = new GenericFileDeleteProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
             return strategy;
-        } else if (move || preMove) {
-            GenericFileRenameProcessStrategy<File> strategy = new GenericFileRenameProcessStrategy<File>();
-            strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
-            if (move) {
-                strategy.setCommitRenamer(new GenericFileDefaultRenamer<File>(moveNamePrefix, moveNamePostfix));
-            }
-            if (preMove) {
-                strategy.setBeginRenamer(new GenericFileDefaultRenamer<File>(preMoveNamePrefix, preMoveNamePostfix));
-            }
-            return strategy;
         } else if (moveExpression != null || preMoveExpression != null) {
             GenericFileRenameProcessStrategy<File> strategy = new GenericFileRenameProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
@@ -79,7 +64,8 @@
             // default strategy will move files in a .camel/ subfolder
             GenericFileRenameProcessStrategy<File> strategy = new GenericFileRenameProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
-            strategy.setCommitRenamer(new GenericFileDefaultRenamer<File>(".camel/", ""));
+            Expression exp = FileLanguage.file(".camel/${file:name}");
+            strategy.setCommitRenamer(new GenericFileExpressionRenamer<File>(exp));
             return strategy;
         }
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewMarkerFileExclusiveReadLockStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewMarkerFileExclusiveReadLockStrategy.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewMarkerFileExclusiveReadLockStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/NewMarkerFileExclusiveReadLockStrategy.java Sun Feb  8 13:13:14 2009
@@ -24,7 +24,6 @@
 
 import org.apache.camel.Exchange;
 import org.apache.camel.component.file.GenericFile;
-import org.apache.camel.component.file.GenericFileExchange;
 import org.apache.camel.component.file.GenericFileExclusiveReadLockStrategy;
 import org.apache.camel.component.file.GenericFileOperations;
 import org.apache.camel.component.file.NewFileComponent;
@@ -34,19 +33,17 @@
 import org.apache.commons.logging.LogFactory;
 
 /**
- *
+ * Acquires read lock to the given file using a marker file so other Camel consumers wont acquire the same file.
+ * This is the default behaviour in Camel 1.x.
  */
 public class NewMarkerFileExclusiveReadLockStrategy implements GenericFileExclusiveReadLockStrategy<File> {
     private static final transient Log LOG = LogFactory.getLog(NewMarkerFileExclusiveReadLockStrategy.class);
 
-    private GenericFileRenamer<File> lockFileRenamer = new GenericFileDefaultRenamer<File>("", NewFileComponent.DEFAULT_LOCK_FILE_POSTFIX);
-
     @SuppressWarnings("unchecked")
     public boolean acquireExclusiveReadLock(GenericFileOperations<File> fileGenericFileOperations,
                                             GenericFile<File> file, Exchange exchange) throws Exception {
 
-        GenericFile newFile = lockFileRenamer.renameFile((GenericFileExchange<File>) exchange, file);
-        String lockFileName = newFile.getAbsoluteFileName();
+        String lockFileName = file.getAbsoluteFileName() + NewFileComponent.DEFAULT_LOCK_FILE_POSTFIX;
         if (LOG.isTraceEnabled()) {
             LOG.trace("Locking the file: " + file + " using the lock file name: " + lockFileName);
         }
@@ -54,8 +51,8 @@
         FileChannel channel = new RandomAccessFile(lockFileName, "rw").getChannel();
         FileLock lock = channel.lock();
         if (lock != null) {
-            exchange.setProperty("org.apache.camel.file.marker.lock", lock);
-            exchange.setProperty("org.apache.camel.file.marker.filename", lockFileName);
+            exchange.setProperty("CamelFileLock", lock);
+            exchange.setProperty("CamelFileLockName", lockFileName);
             return true;
         } else {
             return false;
@@ -64,8 +61,8 @@
 
     public void releaseExclusiveReadLock(GenericFileOperations<File> fileGenericFileOperations,
                                          GenericFile<File> fileGenericFile, Exchange exchange) throws Exception {
-        FileLock lock = ExchangeHelper.getMandatoryProperty(exchange, "org.apache.camel.file.marker.lock", FileLock.class);
-        String lockFileName = ExchangeHelper.getMandatoryProperty(exchange, "org.apache.camel.file.marker.filename", String.class);
+        FileLock lock = ExchangeHelper.getMandatoryProperty(exchange, "CamelFileLock", FileLock.class);
+        String lockFileName = ExchangeHelper.getMandatoryProperty(exchange, "CamelFileLockName", String.class);
         Channel channel = lock.channel();
 
         if (LOG.isTraceEnabled()) {

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginAndCommitRenameStrategyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginAndCommitRenameStrategyTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginAndCommitRenameStrategyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginAndCommitRenameStrategyTest.java Sun Feb  8 13:13:14 2009
@@ -50,7 +50,7 @@
 
     public void testIllegalOptions() throws Exception {
         try {
-            context.getEndpoint("newfile://target?moveNamePrefix=../done/&delete=true").createConsumer(new Processor() {
+            context.getEndpoint("newfile://target?moveExpression=../done/${file:name}&delete=true").createConsumer(new Processor() {
                 public void process(Exchange exchange) throws Exception {
                 }
             });
@@ -73,7 +73,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("newfile://target/reports?preMoveNamePrefix=../inprogress/&moveNamePrefix=../done/&consumer.delay=5000")
+                from("newfile://target/reports?preMoveExpression=../inprogress/${file:name}&moveExpression=../done/${file:name}&consumer.delay=5000")
                         .process(new Processor() {
                             public void process(Exchange exchange) throws Exception {
                                 GenericFileExchange<File> fe = (GenericFileExchange<File>) exchange;

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginRenameStrategyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginRenameStrategyTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginRenameStrategyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerBeginRenameStrategyTest.java Sun Feb  8 13:13:14 2009
@@ -70,7 +70,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("newfile://target/reports?preMoveNamePrefix=../inprogress/&consumer.delay=5000")
+                from("newfile://target/reports?preMoveExpression=../inprogress/${file:name}&consumer.delay=5000")
                         .process(new Processor() {
                             public void process(Exchange exchange) throws Exception {
                                 GenericFileExchange<File> fe = (GenericFileExchange<File>) exchange;

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerCommitRenameStrategyTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerCommitRenameStrategyTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerCommitRenameStrategyTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerCommitRenameStrategyTest.java Sun Feb  8 13:13:14 2009
@@ -76,7 +76,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("file://target/reports?moveNamePrefix=../done/&consumer.delay=5000").to("mock:report");
+                from("file://target/reports?moveExpression=../done/${file:name}&consumer.delay=5000").to("mock:report");
             }
         };
     }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java Sun Feb  8 13:13:14 2009
@@ -49,7 +49,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("file://target/idempotent/?idempotent=true&idempotentRepository=#myRepo&moveNamePrefix=done/").to("mock:result");
+                from("file://target/idempotent/?idempotent=true&idempotentRepository=#myRepo&moveExpression=done/${file:name}").to("mock:result");
             }
         };
     }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentTest.java Sun Feb  8 13:13:14 2009
@@ -38,7 +38,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("file://target/idempotent/?idempotent=true&moveNamePrefix=done/").to("mock:result");
+                from("file://target/idempotent/?idempotent=true&moveExpression=done/${file:name}").to("mock:result");
             }
         };
     }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileRouteTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileRouteTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileRouteTest.java Sun Feb  8 13:13:14 2009
@@ -68,7 +68,7 @@
         }
 
         public void process(Exchange exchange) {
-            locks.add(exchange.getProperty("org.apache.camel.file.marker.filename", String.class));
+            locks.add(exchange.getProperty("CamelFileLockName", String.class));
         }
     }
 }

Modified: camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileConsumer.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileConsumer.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileConsumer.java (original)
+++ camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileConsumer.java Sun Feb  8 13:13:14 2009
@@ -50,7 +50,7 @@
         try {
             if (((RemoteFileOperations) operations).isConnected()) {
                 loggedIn = false;
-                log.debug("Disconnecting from " + remoteServer());
+                log.debug("Disconnecting from: " + remoteServer());
                 ((RemoteFileOperations) operations).disconnect();
             }
         } catch (GenericFileOperationFailedException e) {
@@ -62,11 +62,11 @@
     protected void connectIfNecessary() throws IOException {
         if (!((RemoteFileOperations) operations).isConnected() || !loggedIn) {
             if (log.isDebugEnabled()) {
-                log.debug("Not connected/logged in, connecting to " + remoteServer());
+                log.debug("Not connected/logged in, connecting to: " + remoteServer());
             }
             loggedIn = ((RemoteFileOperations) operations).connect((RemoteFileConfiguration) endpoint.getConfiguration());
             if (loggedIn) {
-                log.info("Connected and logged in to " + remoteServer());
+                log.info("Connected and logged in to: " + remoteServer());
             }
         }
     }

Modified: camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java (original)
+++ camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java Sun Feb  8 13:13:14 2009
@@ -61,8 +61,8 @@
         afterPropertiesSet();
         RemoteFileConsumer<T> consumer = buildConsumer(processor, (RemoteFileOperations<T>) operations);
 
-        if (isDelete() && (getMoveNamePrefix() != null || getMoveNamePostfix() != null || getMoveExpression() != null)) {
-            throw new IllegalArgumentException("You cannot both set delete=true and either a (moveNamePrefix, moveNamePostfix or moveExpression) option");
+        if (isDelete() && getMoveExpression() != null) {
+            throw new IllegalArgumentException("You cannot both set delete=true and moveExpression options");
         }
         // if noop=true then idempotent should also be configured
         if (isNoop() && !isIdempotent()) {

Modified: camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileProducer.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileProducer.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileProducer.java (original)
+++ camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileProducer.java Sun Feb  8 13:13:14 2009
@@ -36,6 +36,7 @@
         super(endpoint, operations);
     }
 
+    @SuppressWarnings("unchecked")
     public void process(Exchange exchange) throws Exception {
         GenericFileExchange remoteExchange = (GenericFileExchange) getEndpoint().createExchange(exchange);
         processExchange(remoteExchange);
@@ -49,9 +50,9 @@
         loggedIn = false;
         if (isStopping() || isStopped()) {
             // if we are stopping then ignore any exception during a poll
-            log.debug("Exception occured during stopping. " + exception.getMessage());
+            log.debug("Exception occured during stopping: " + exception.getMessage());
         } else {
-            log.debug("Exception occured during processing.", exception);
+            log.debug("Exception occured during processing. ", exception);
             disconnect();
             // Rethrow to signify that we didn't poll
             throw exception;
@@ -61,7 +62,7 @@
     public void disconnect() throws IOException {
         loggedIn = false;
         if (log.isDebugEnabled()) {
-            log.debug("Disconnecting from " + getEndpoint());
+            log.debug("Disconnecting from: " + getEndpoint());
         }
         ((RemoteFileOperations) operations).disconnect();
     }
@@ -89,7 +90,7 @@
         try {
             disconnect();
         } catch (Exception e) {
-            log.debug("Exception occured during disconnecting from " + getEndpoint() + " " + e.getMessage());
+            log.debug("Exception occured during disconnecting from: " + getEndpoint() + " " + e.getMessage());
         }
         super.doStop();
     }
@@ -106,7 +107,7 @@
             if (!loggedIn) {
                 return;
             }
-            log.info("Connected and logged in to " + getEndpoint());
+            log.info("Connected and logged in to: " + getEndpoint());
         }
     }
 

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePostfixTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePostfixTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePostfixTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePostfixTest.java Sun Feb  8 13:13:14 2009
@@ -32,7 +32,7 @@
 
     private String getFtpUrl() {
         return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false"
-                + "&moveNamePostfix=.old&consumer.delay=5000";
+                + "&moveExpression=${file:name}.old&consumer.delay=5000";
     }
 
     @Override

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePrefixTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePrefixTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePrefixTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFilePrefixTest.java Sun Feb  8 13:13:14 2009
@@ -32,7 +32,7 @@
 
     private String getFtpUrl() {
         return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false&consumer.delay=5000"
-                + "&moveNamePrefix=done/";
+                + "&moveExpression=done/${file:name}";
     }
 
     @Override

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFileTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFileTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFileTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpMoveFileTest.java Sun Feb  8 13:13:14 2009
@@ -32,7 +32,7 @@
 
     private String getFtpUrl() {
         return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false"
-                + "&moveNamePrefix=done/sub2/&moveNamePostfix=.old&consumer.delay=5000";
+                + "&moveExpression=done/sub2/${file:name}.old&consumer.delay=5000";
     }
 
     @Override

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePostfixTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePostfixTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePostfixTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePostfixTest.java Sun Feb  8 13:13:14 2009
@@ -32,7 +32,7 @@
 
     private String getFtpUrl() {
         return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false"
-                + "&preMoveNamePostfix=.old";
+                + "&preMoveExpression=${file:name}.old";
     }
 
     @Override

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePrefixTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePrefixTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePrefixTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FromFtpPreMoveFilePrefixTest.java Sun Feb  8 13:13:14 2009
@@ -32,7 +32,7 @@
 
     private String getFtpUrl() {
         return "ftp://admin@localhost:" + getPort() + "/movefile?password=admin&binary=false&consumer.delay=5000"
-                + "&preMoveNamePrefix=done/";
+                + "&preMoveExpression=done/${file:name}";
     }
 
     @Override

Modified: camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FtpIllegalOptionsTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FtpIllegalOptionsTest.java?rev=742087&r1=742086&r2=742087&view=diff
==============================================================================
--- camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FtpIllegalOptionsTest.java (original)
+++ camel/trunk/components/camel-ftp/src/test/java/org/apache/camel/component/file/remote/FtpIllegalOptionsTest.java Sun Feb  8 13:13:14 2009
@@ -34,17 +34,7 @@
         }
 
         try {
-            context.getEndpoint("file://target?moveNamePrefix=../done/&delete=true").createConsumer(new Processor() {
-                public void process(Exchange exchange) throws Exception {
-                }
-            });
-            fail("Should have thrown an exception");
-        } catch (IllegalArgumentException e) {
-            // ok
-        }
-
-        try {
-            context.getEndpoint("file://target?moveNamePostfix=.bak&delete=true").createConsumer(new Processor() {
+            context.getEndpoint("file://target?moveExpression=../done/${file:name}&delete=true").createConsumer(new Processor() {
                 public void process(Exchange exchange) throws Exception {
                 }
             });