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 2013/01/30 18:56:21 UTC

svn commit: r1440560 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/ main/java/org/apache/camel/component/file/ test/java/org/apache/camel/component/file/

Author: davsclaus
Date: Wed Jan 30 17:56:20 2013
New Revision: 1440560

URL: http://svn.apache.org/viewvc?rev=1440560&view=rev
Log:
CAMEL-5269: Allow file/ftp producer to use overrule filename header to use for writing file name, and throwaway after use, and preserve original filename header. Makes it easier to keep same file name and write to an endpoint using a different name.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleExpressionTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleOnlyOnceTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleTest.java
      - copied, changed from r1440442, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceAppendTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileProducer.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java?rev=1440560&r1=1440559&r2=1440560&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java Wed Jan 30 17:56:20 2013
@@ -154,7 +154,8 @@ public interface Exchange {
 
     String NOTIFY_EVENT = "CamelNotifyEvent";
 
-    String ON_COMPLETION = "CamelOnCompletion";
+    String ON_COMPLETION      = "CamelOnCompletion";
+    String OVERRULE_FILE_NAME = "CamelOverruleFileName";
 
     String PARENT_UNIT_OF_WORK = "CamelParentUnitOfWork";
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileProducer.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileProducer.java?rev=1440560&r1=1440559&r2=1440560&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileProducer.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/GenericFileProducer.java Wed Jan 30 17:56:20 2013
@@ -58,6 +58,10 @@ public class GenericFileProducer<T> exte
     }
 
     public void process(Exchange exchange) throws Exception {
+        // store any existing file header which we want to keep and propagate
+        final String existing = exchange.getIn().getHeader(Exchange.FILE_NAME, String.class);
+
+        // create the target file name
         String target = createFileName(exchange);
 
         // use lock for same file name to avoid concurrent writes to the same file
@@ -78,6 +82,10 @@ public class GenericFileProducer<T> exte
             // do not remove as the locks cache has an upper bound
             // this ensure the locks is appropriate reused
             lock.unlock();
+            // and remove the write file name header as we only want to use it once (by design)
+            exchange.getIn().removeHeader(Exchange.OVERRULE_FILE_NAME);
+            // and restore existing file name
+            exchange.getIn().setHeader(Exchange.FILE_NAME, existing);
         }
     }
 
@@ -256,7 +264,14 @@ public class GenericFileProducer<T> exte
     public String createFileName(Exchange exchange) {
         String answer;
 
-        String name = exchange.getIn().getHeader(Exchange.FILE_NAME, String.class);
+        // overrule takes precedence
+        String overrule = exchange.getIn().getHeader(Exchange.OVERRULE_FILE_NAME, String.class);
+        String name = overrule == null ? exchange.getIn().getHeader(Exchange.FILE_NAME, String.class) : overrule;
+
+        // if we have an overrule then override the existing header to use the overrule computed name from this point forward
+        if (overrule != null) {
+            exchange.getIn().setHeader(Exchange.FILE_NAME, name);
+        }
 
         // expression support
         Expression expression = endpoint.getFileName();

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleExpressionTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleExpressionTest.java?rev=1440560&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleExpressionTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleExpressionTest.java Wed Jan 30 17:56:20 2013
@@ -0,0 +1,76 @@
+/**
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * Unit test to verify the writeFileName option
+ */
+public class FileProduceOverruleExpressionTest extends ContextTestSupport {
+
+    public void testNoOverrule() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedHeaderReceived(Exchange.FILE_NAME, "hello.txt");
+        mock.expectedFileExists("target/write/copy-of-hello.txt", "Hello World");
+
+        template.sendBodyAndHeader("direct:start", "Hello World", Exchange.FILE_NAME, "hello.txt");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testOverrule() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedHeaderReceived(Exchange.FILE_NAME, "hello.txt");
+        mock.message(0).header(Exchange.OVERRULE_FILE_NAME).isNull();
+        mock.expectedFileExists("target/write/copy-of-overruled.txt", "Hello World");
+
+        Map<String, Object> map = new HashMap<String, Object>();
+        map.put(Exchange.FILE_NAME, "hello.txt");
+        // this header should overrule the endpoint configuration
+        map.put(Exchange.OVERRULE_FILE_NAME, "overruled.txt");
+
+        template.sendBodyAndHeaders("direct:start", "Hello World", map);
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/write");
+        super.setUp();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                from("direct:start")
+                    .to("file://target/write?fileName=copy-of-${file:name}", "mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

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

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

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleOnlyOnceTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleOnlyOnceTest.java?rev=1440560&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleOnlyOnceTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleOnlyOnceTest.java Wed Jan 30 17:56:20 2013
@@ -0,0 +1,68 @@
+/**
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * Unit test to verify the overrule filename header
+ */
+public class FileProduceOverruleOnlyOnceTest extends ContextTestSupport {
+
+    public void testBoth() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedHeaderReceived(Exchange.FILE_NAME, "hello.txt");
+        mock.message(0).header(Exchange.OVERRULE_FILE_NAME).isNull();
+        mock.expectedFileExists("target/write/ruled.txt", "Hello World");
+        mock.expectedFileExists("target/again/hello.txt", "Hello World");
+
+        Map<String, Object> map = new HashMap<String, Object>();
+        map.put(Exchange.FILE_NAME, "hello.txt");
+        // this header should overrule the endpoint configuration
+        map.put(Exchange.OVERRULE_FILE_NAME, "ruled.txt");
+
+        template.sendBodyAndHeaders("direct:start", "Hello World", map);
+
+        assertMockEndpointsSatisfied();
+    }
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/write");
+        deleteDirectory("target/again");
+        super.setUp();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                from("direct:start")
+                    .to("file://target/write")
+                    .to("file://target/again", "mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

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

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

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleTest.java (from r1440442, camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceAppendTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceAppendTest.java&r1=1440442&r2=1440560&rev=1440560&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceAppendTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileProduceOverruleTest.java Wed Jan 30 17:56:20 2013
@@ -16,32 +16,62 @@
  */
 package org.apache.camel.component.file;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 
 /**
- * Unit test to verify the append option
+ * Unit test to verify the overrule filename header
  */
-public class FileProduceAppendTest extends ContextTestSupport {
+public class FileProduceOverruleTest extends ContextTestSupport {
+
+    public void testNoOverrule() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedHeaderReceived(Exchange.FILE_NAME, "hello.txt");
+        mock.expectedFileExists("target/write/hello.txt", "Hello World");
+
+        template.sendBodyAndHeader("direct:start", "Hello World", Exchange.FILE_NAME, "hello.txt");
+
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testOnlyOverrule() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.message(0).header(Exchange.FILE_NAME).isNull();
+        mock.expectedFileExists("target/write/overrule.txt", "Hello World");
+
+        template.sendBodyAndHeader("direct:start", "Hello World", Exchange.OVERRULE_FILE_NAME, "overrule.txt");
+
+        assertMockEndpointsSatisfied();
+    }
 
-    public void testAppendText() throws Exception {
+    public void testBoth() throws Exception {
         MockEndpoint mock = getMockEndpoint("mock:result");
         mock.expectedMessageCount(1);
-        mock.expectedFileExists("target/test-file-append/hello.txt", "Hello World");
+        mock.expectedHeaderReceived(Exchange.FILE_NAME, "hello.txt");
+        mock.message(0).header(Exchange.OVERRULE_FILE_NAME).isNull();
+        mock.expectedFileExists("target/write/ruled.txt", "Hello World");
+
+        Map<String, Object> map = new HashMap<String, Object>();
+        map.put(Exchange.FILE_NAME, "hello.txt");
+        // this header should overrule the endpoint configuration
+        map.put(Exchange.OVERRULE_FILE_NAME, "ruled.txt");
 
-        template.sendBody("direct:start", " World");
+        template.sendBodyAndHeaders("direct:start", "Hello World", map);
 
         assertMockEndpointsSatisfied();
     }
 
     @Override
     protected void setUp() throws Exception {
-        deleteDirectory("target/test-file-append");
+        deleteDirectory("target/write");
         super.setUp();
-        template.sendBodyAndHeader("file://target/test-file-append", "Hello", Exchange.FILE_NAME, "hello.txt");
-        template.sendBodyAndHeader("file://target/test-file-append", " World", Exchange.FILE_NAME, "world.txt");
     }
 
     @Override
@@ -49,8 +79,7 @@ public class FileProduceAppendTest exten
         return new RouteBuilder() {
             public void configure() {
                 from("direct:start")
-                    .setHeader(Exchange.FILE_NAME, constant("hello.txt"))
-                    .to("file://target/test-file-append?fileExist=Append", "mock:result");
+                    .to("file://target/write", "mock:result");
             }
         };
     }