You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by ni...@apache.org on 2010/06/30 08:54:27 UTC

svn commit: r959214 - 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: ningjiang
Date: Wed Jun 30 06:54:26 2010
New Revision: 959214

URL: http://svn.apache.org/viewvc?rev=959214&view=rev
Log:
CAMEL-2879 Support using delete=true and moveFailed option at the same time

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java   (contents, props changed)
      - copied, changed from r959184, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailureTest.java
Removed:
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailureTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java
    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/GenericFileProcessStrategySupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileRenameProcessStrategy.java
    camel/trunk/components/camel-ftp/src/main/java/org/apache/camel/component/file/remote/RemoteFileEndpoint.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java?rev=959214&r1=959213&r2=959214&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java Wed Jun 30 06:54:26 2010
@@ -67,8 +67,8 @@ public class FileEndpoint extends Generi
 
         FileConsumer result = new FileConsumer(this, processor, operations);
 
-        if (isDelete() && (getMove() != null || getMoveFailed() != null)) {
-            throw new IllegalArgumentException("You cannot set both delete=true and move or moveFailed options");
+        if (isDelete() && getMove() != null) {
+            throw new IllegalArgumentException("You cannot set both delete=true and move options");
         }
 
         // if noop=true then idempotent should also be configured

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=959214&r1=959213&r2=959214&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 Jun 30 06:54:26 2010
@@ -48,6 +48,11 @@ public final class FileProcessStrategyFa
         } else if (isDelete) {
             GenericFileDeleteProcessStrategy<File> strategy = new GenericFileDeleteProcessStrategy<File>();
             strategy.setExclusiveReadLockStrategy(getExclusiveReadLockStrategy(params));
+            if (moveFailedExpression != null) {
+                GenericFileExpressionRenamer<File> renamer = new GenericFileExpressionRenamer<File>();
+                renamer.setExpression(moveFailedExpression);
+                strategy.setFailureRenamer(renamer);
+            }
             return strategy;
         } else if (isMove) {
             GenericFileRenameProcessStrategy<File> strategy = new GenericFileRenameProcessStrategy<File>();

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=959214&r1=959213&r2=959214&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 Jun 30 06:54:26 2010
@@ -24,6 +24,8 @@ import org.apache.camel.component.file.G
 
 public class GenericFileDeleteProcessStrategy<T> extends GenericFileProcessStrategySupport<T> {
 
+    private GenericFileRenamer<T> failureRenamer;
+    
     @Override
     public void commit(GenericFileOperations<T> operations, GenericFileEndpoint<T> endpoint, Exchange exchange, GenericFile<T> file) throws Exception {
         // must invoke super
@@ -59,5 +61,26 @@ public class GenericFileDeleteProcessStr
             throw new GenericFileOperationFailedException("Cannot delete file: " + file);
         }
     }
+    
+    @Override
+    public void rollback(GenericFileOperations<T> operations, GenericFileEndpoint<T> endpoint, Exchange exchange, GenericFile<T> file) throws Exception {
+        // must invoke super
+        super.rollback(operations, endpoint, exchange, file);
+
+        // moved the failed file if specifying the moveFailed option
+        if (failureRenamer != null) {
+            GenericFile<T> newName = failureRenamer.renameFile(exchange, file);
+            renameFile(operations, file, newName);
+        }
+    }
+    
+    public GenericFileRenamer<T> getFailureRenamer() {
+        return failureRenamer;
+    }
+
+    public void setFailureRenamer(GenericFileRenamer<T> failureRenamer) {
+        this.failureRenamer = failureRenamer;
+    }
+
 
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategySupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategySupport.java?rev=959214&r1=959213&r2=959214&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategySupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/strategy/GenericFileProcessStrategySupport.java Wed Jun 30 06:54:26 2010
@@ -17,11 +17,13 @@
 package org.apache.camel.component.file.strategy;
 
 import java.io.File;
+import java.io.IOException;
 
 import org.apache.camel.Exchange;
 import org.apache.camel.component.file.GenericFile;
 import org.apache.camel.component.file.GenericFileEndpoint;
 import org.apache.camel.component.file.GenericFileExclusiveReadLockStrategy;
+import org.apache.camel.component.file.GenericFileOperationFailedException;
 import org.apache.camel.component.file.GenericFileOperations;
 import org.apache.camel.component.file.GenericFileProcessStrategy;
 import org.apache.camel.util.FileUtil;
@@ -74,6 +76,32 @@ public abstract class GenericFileProcess
     public void setExclusiveReadLockStrategy(GenericFileExclusiveReadLockStrategy<T> exclusiveReadLockStrategy) {
         this.exclusiveReadLockStrategy = exclusiveReadLockStrategy;
     }
+    
+    protected GenericFile<T> renameFile(GenericFileOperations<T> operations, GenericFile<T> from, GenericFile<T> to) throws IOException {
+        // deleting any existing files before renaming
+        try {
+            operations.deleteFile(to.getAbsoluteFilePath());
+        } catch (GenericFileOperationFailedException e) {
+            // ignore the file does not exists
+        }
+        
+        // make parent folder if missing
+        boolean mkdir = operations.buildDirectory(to.getParent(), to.isAbsolute());
+        
+        if (!mkdir) {
+            throw new GenericFileOperationFailedException("Cannot create directory: " + to.getParent() + " (could be because of denied permissions)");
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Renaming file: " + from + " to: " + to);
+        }
+        boolean renamed = operations.renameFile(from.getAbsoluteFilePath(), to.getAbsoluteFilePath());
+        if (!renamed) {
+            throw new GenericFileOperationFailedException("Cannot rename file: " + from + " to: " + to);
+        }
+
+        return to;
+    }
 
     private void deleteLocalWorkFile(Exchange exchange) {
         // delete local work file, if it was used (eg by ftp component)

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=959214&r1=959213&r2=959214&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 Wed Jun 30 06:54:26 2010
@@ -73,32 +73,6 @@ public class GenericFileRenameProcessStr
         }
     }
 
-    private GenericFile<T> renameFile(GenericFileOperations<T> operations, GenericFile<T> from, GenericFile<T> to) throws IOException {
-        // deleting any existing files before renaming
-        try {
-            operations.deleteFile(to.getAbsoluteFilePath());
-        } catch (GenericFileOperationFailedException e) {
-            // ignore the file does not exists
-        }
-        
-        // make parent folder if missing
-        boolean mkdir = operations.buildDirectory(to.getParent(), to.isAbsolute());
-        
-        if (!mkdir) {
-            throw new GenericFileOperationFailedException("Cannot create directory: " + to.getParent() + " (could be because of denied permissions)");
-        }
-
-        if (log.isDebugEnabled()) {
-            log.debug("Renaming file: " + from + " to: " + to);
-        }
-        boolean renamed = operations.renameFile(from.getAbsoluteFilePath(), to.getAbsoluteFilePath());
-        if (!renamed) {
-            throw new GenericFileOperationFailedException("Cannot rename file: " + from + " to: " + to);
-        }
-
-        return to;
-    }
-
     public GenericFileRenamer<T> getBeginRenamer() {
         return beginRenamer;
     }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java?rev=959214&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java Wed Jun 30 06:54:26 2010
@@ -0,0 +1,67 @@
+/**
+ * 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 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$
+ */
+public class FileConsumerDeleteAndFailureTest extends ContextTestSupport {
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/failed");
+        super.setUp();
+    }
+
+    public void testMoveFailed() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceived("Hello World IS processed!");
+        
+        mock.expectedFileExists("target/failed/error/bye.txt", "Kabom");
+
+        template.sendBodyAndHeader("file://target/failed", "Hello World", Exchange.FILE_NAME, "hello.txt");
+        template.sendBodyAndHeader("file://target/failed", "Kabom", Exchange.FILE_NAME, "bye.txt");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                onException(IllegalArgumentException.class).useOriginalMessage().to("file://target/failed/error");
+                from("file://target/failed?delete=true")
+                    .setBody(simple("${body} IS processed!"))
+                    .process(new Processor() {
+                        public void process(Exchange exchange) throws Exception {
+                            String body = exchange.getIn().getBody(String.class);
+                            if (body != null && body.startsWith("Kabom")) {
+                                throw new IllegalArgumentException("Forced");
+                            }
+                        }
+                    }).to("mock:result");                
+            }
+        };
+    }
+}

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndFailureTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java (from r959184, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailureTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailureTest.java&r1=959184&r2=959214&rev=959214&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailureTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java Wed Jun 30 06:54:26 2010
@@ -25,48 +25,14 @@ import org.apache.camel.component.mock.M
 /**
  * @version $Revision$
  */
-public class FileConsumerDeleteAndMoveFailureTest extends ContextTestSupport {
-
-    @Override
-    protected void setUp() throws Exception {
-        deleteDirectory("target/failed");
-        super.setUp();
-    }
-
-    public void testMoveFailed() throws Exception {
-        MockEndpoint mock = getMockEndpoint("mock:result");
-        mock.expectedBodiesReceived("Hello World IS processed!");
-        
-        mock.expectedFileExists("target/failed/error/bye.txt", "Kabom");
-
-        template.sendBodyAndHeader("file://target/failed", "Hello World", Exchange.FILE_NAME, "hello.txt");
-        template.sendBodyAndHeader("file://target/failed", "Kabom", Exchange.FILE_NAME, "bye.txt");
-
-        assertMockEndpointsSatisfied();
-    }
-    
-    public void testDeletAndMoveFailedOption() throws Exception {
-        try {
-            context.addRoutes(new RouteBuilder() {
-                public void configure() throws Exception {
-                    from("file://target/test?delete=true&moveFailed=target/failed/error").to("mock:failed");
-                }
-            });
-            fail("Expect an exception here");
-        } catch (IllegalArgumentException ex) {
-            // expect the error here
-            ex.getMessage().startsWith("You cannot set both deleted=true and move");
-        }
-        
-    }
+public class FileConsumerDeleteAndMoveFailedTest extends FileConsumerDeleteAndFailureTest {
 
     @Override
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                onException(IllegalArgumentException.class).useOriginalMessage().to("file://target/failed/error");
-                from("file://target/failed?delete=true")
+                from("file://target/failed?delete=true&moveFailed=error")
                     .setBody(simple("${body} IS processed!"))
                     .process(new Processor() {
                         public void process(Exchange exchange) throws Exception {
@@ -75,7 +41,7 @@ public class FileConsumerDeleteAndMoveFa
                                 throw new IllegalArgumentException("Forced");
                             }
                         }
-                    }).to("mock:result");
+                    }).to("mock:result");                
             }
         };
     }

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerDeleteAndMoveFailedTest.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

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=959214&r1=959213&r2=959214&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 Wed Jun 30 06:54:26 2010
@@ -63,8 +63,8 @@ public abstract class RemoteFileEndpoint
         afterPropertiesSet();
         RemoteFileConsumer<T> consumer = buildConsumer(processor);
 
-        if (isDelete() && (getMove() != null || getMoveFailed() != null)) {
-            throw new IllegalArgumentException("You cannot both set delete=true and move or moveFailed options");
+        if (isDelete() && getMove() != null) {
+            throw new IllegalArgumentException("You cannot both set delete=true and move options");
         }
         // if noop=true then idempotent should also be configured
         if (isNoop() && !isIdempotent()) {