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 2016/05/06 16:17:38 UTC

camel git commit: CAMEL-9067: File consumer allow to filter using simple predicates for directory and file separated. This allows some filtering to be done without writing code.

Repository: camel
Updated Branches:
  refs/heads/master ab31bc622 -> 406bf89bb


CAMEL-9067: File consumer allow to filter using simple predicates for directory and file separated. This allows some filtering to be done without writing code.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/406bf89b
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/406bf89b
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/406bf89b

Branch: refs/heads/master
Commit: 406bf89bb615c8dff610f5e0fa49121515a2a6c8
Parents: ab31bc6
Author: Claus Ibsen <da...@apache.org>
Authored: Fri May 6 18:05:38 2016 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Fri May 6 18:17:23 2016 +0200

----------------------------------------------------------------------
 .../component/file/GenericFileConsumer.java     | 18 +++++
 .../component/file/GenericFileEndpoint.java     | 51 +++++++++++++-
 .../file/FileConsumerFilterDirectoryTest.java   | 72 ++++++++++++++++++++
 .../file/FileConsumerFilterFileTest.java        | 70 +++++++++++++++++++
 components/camel-ftp/src/main/docs/ftp.adoc     |  6 +-
 5 files changed, 213 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/406bf89b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileConsumer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/file/GenericFileConsumer.java b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileConsumer.java
index f28aee5..399dabd 100644
--- a/camel-core/src/main/java/org/apache/camel/component/file/GenericFileConsumer.java
+++ b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileConsumer.java
@@ -604,6 +604,15 @@ public abstract class GenericFileConsumer<T> extends ScheduledBatchPollingConsum
             }
         }
 
+        if (isDirectory && endpoint.getFilterDirectory() != null) {
+            // create a dummy exchange as Exchange is needed for expression evaluation
+            Exchange dummy = endpoint.createExchange(file);
+            boolean matches = endpoint.getFilterDirectory().matches(dummy);
+            if (!matches) {
+                return false;
+            }
+        }
+
         // directories are regarded as matched if filter accepted them
         if (isDirectory) {
             return true;
@@ -631,6 +640,15 @@ public abstract class GenericFileConsumer<T> extends ScheduledBatchPollingConsum
             }
         }
 
+        if (endpoint.getFilterFile() != null) {
+            // create a dummy exchange as Exchange is needed for expression evaluation
+            Exchange dummy = endpoint.createExchange(file);
+            boolean matches = endpoint.getFilterFile().matches(dummy);
+            if (!matches) {
+                return false;
+            }
+        }
+
         // if done file name is enabled, then the file is only valid if a done file exists
         if (endpoint.getDoneFileName() != null) {
             // done file must be in same path as the file

http://git-wip-us.apache.org/repos/asf/camel/blob/406bf89b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
index 518d127..064fc60 100644
--- a/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/file/GenericFileEndpoint.java
@@ -18,14 +18,11 @@ package org.apache.camel.component.file;
 
 import java.io.IOException;
 import java.lang.reflect.Method;
-import java.nio.file.attribute.PosixFilePermission;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Component;
@@ -34,6 +31,7 @@ import org.apache.camel.Expression;
 import org.apache.camel.ExpressionIllegalSyntaxException;
 import org.apache.camel.LoggingLevel;
 import org.apache.camel.Message;
+import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
 import org.apache.camel.impl.ScheduledPollEndpoint;
 import org.apache.camel.processor.idempotent.MemoryIdempotentRepository;
@@ -139,6 +137,10 @@ public abstract class GenericFileEndpoint<T> extends ScheduledPollEndpoint imple
     protected IdempotentRepository<String> idempotentRepository;
     @UriParam(label = "consumer,filter")
     protected GenericFileFilter<T> filter;
+    @UriParam(label = "consumer,filter", javaType = "java.lang.String")
+    protected Predicate filterDirectory;
+    @UriParam(label = "consumer,filter", javaType = "java.lang.String")
+    protected Predicate filterFile;
     @UriParam(label = "consumer,filter", defaultValue = "true")
     protected boolean antFilterCaseSensitive = true;
     protected volatile AntPathMatcherGenericFileFilter<T> antFilter;
@@ -462,6 +464,44 @@ public abstract class GenericFileEndpoint<T> extends ScheduledPollEndpoint imple
         this.moveFailed = createFileLanguageExpression(expression);
     }
 
+    public Predicate getFilterDirectory() {
+        return filterDirectory;
+    }
+
+    /**
+     * Filters the directory based on Simple language.
+     * For example to filter on current date, you can use a simple date pattern such as ${date:now:yyyMMdd}
+     */
+    public void setFilterDirectory(Predicate filterDirectory) {
+        this.filterDirectory = filterDirectory;
+    }
+
+    /**
+     * @see #setFilterDirectory(Predicate)
+     */
+    public void setFilterDirectory(String expression) {
+        this.filterDirectory = createFileLanguagePredicate(expression);
+    }
+
+    public Predicate getFilterFile() {
+        return filterFile;
+    }
+
+    /**
+     * Filters the file based on Simple language.
+     * For example to filter on file size, you can use ${file:size} > 5000
+     */
+    public void setFilterFile(Predicate filterFile) {
+        this.filterFile = filterFile;
+    }
+
+    /**
+     * @see #setFilterFile(Predicate)
+     */
+    public void setFilterFile(String expression) {
+        this.filterFile = createFileLanguagePredicate(expression);
+    }
+
     public Expression getPreMove() {
         return preMove;
     }
@@ -1228,6 +1268,11 @@ public abstract class GenericFileEndpoint<T> extends ScheduledPollEndpoint imple
         return language.createExpression(expression);
     }
 
+    private Predicate createFileLanguagePredicate(String expression) {
+        Language language = getCamelContext().resolveLanguage("file");
+        return language.createPredicate(expression);
+    }
+
     /**
      * Creates the associated name of the done file based on the given file name.
      * <p/>

http://git-wip-us.apache.org/repos/asf/camel/blob/406bf89b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterDirectoryTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterDirectoryTest.java b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterDirectoryTest.java
new file mode 100644
index 0000000..da9e022
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterDirectoryTest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * Unit test for  the filter file option
+ */
+public class FileConsumerFilterDirectoryTest extends ContextTestSupport {
+
+    private String fileUrl = "file://target/filefilter/?recursive=true&filterDirectory=${header.CamelFileNameOnly.length()} > 4";
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/filefilter");
+        deleteDirectory("target/filefilter/foo");
+        deleteDirectory("target/filefilter/barbar");
+        super.setUp();
+    }
+
+    public void testFilterFiles() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(0);
+
+        template.sendBodyAndHeader("file:target/filefilter/foo", "This is a file to be filtered",
+            Exchange.FILE_NAME, "skipme.txt");
+
+        mock.setResultWaitTime(2000);
+        mock.assertIsSatisfied();
+    }
+
+    public void testFilterFilesWithARegularFile() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedBodiesReceived("Hello World");
+
+        template.sendBodyAndHeader("file:target/filefilter/foo", "This is a file to be filtered",
+            Exchange.FILE_NAME, "skipme.txt");
+
+        template.sendBodyAndHeader("file:target/filefilter/barbar", "Hello World",
+            Exchange.FILE_NAME, "hello.txt");
+
+        mock.assertIsSatisfied();
+    }
+
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            public void configure() throws Exception {
+                from(fileUrl).convertBodyTo(String.class).to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/406bf89b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterFileTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterFileTest.java b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterFileTest.java
new file mode 100644
index 0000000..7bbd4ce
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFilterFileTest.java
@@ -0,0 +1,70 @@
+/**
+ * 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.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * Unit test for  the filter file option
+ */
+public class FileConsumerFilterFileTest extends ContextTestSupport {
+
+    private String fileUrl = "file://target/filefilter/?filterFile=${bodyAs(String)} contains 'World'";
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/filefilter");
+        super.setUp();
+    }
+
+    public void testFilterFiles() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(0);
+
+        template.sendBodyAndHeader("file:target/filefilter/", "This is a file to be filtered",
+            Exchange.FILE_NAME, "skipme.txt");
+
+        mock.setResultWaitTime(2000);
+        mock.assertIsSatisfied();
+    }
+
+    public void testFilterFilesWithARegularFile() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedMessageCount(1);
+        mock.expectedBodiesReceived("Hello World");
+
+        template.sendBodyAndHeader("file:target/filefilter/", "This is a file to be filtered",
+            Exchange.FILE_NAME, "skipme.txt");
+
+        template.sendBodyAndHeader("file:target/filefilter/", "Hello World",
+            Exchange.FILE_NAME, "hello.txt");
+
+        mock.assertIsSatisfied();
+    }
+
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            public void configure() throws Exception {
+                from(fileUrl).convertBodyTo(String.class).to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/406bf89b/components/camel-ftp/src/main/docs/ftp.adoc
----------------------------------------------------------------------
diff --git a/components/camel-ftp/src/main/docs/ftp.adoc b/components/camel-ftp/src/main/docs/ftp.adoc
index 56217e7..c54da45 100644
--- a/components/camel-ftp/src/main/docs/ftp.adoc
+++ b/components/camel-ftp/src/main/docs/ftp.adoc
@@ -112,8 +112,9 @@ The FTP component has no options.
 
 
 
+
 // endpoint options: START
-The FTP component supports 100 endpoint options which are listed below:
+The FTP component supports 102 endpoint options which are listed below:
 
 [width="100%",cols="2s,1,1m,1m,5",options="header"]
 |=======================================================================
@@ -180,6 +181,8 @@ The FTP component supports 100 endpoint options which are listed below:
 | eagerMaxMessagesPerPoll | filter | true | boolean | Allows for controlling whether the limit from maxMessagesPerPoll is eager or not. If eager then the limit is during the scanning of files. Where as false would scan all files and then perform sorting. Setting this option to false allows for sorting all files first and then limit the poll. Mind that this requires a higher memory usage as all file details are in memory to perform the sorting.
 | exclude | filter |  | String | Is used to exclude files if filename matches the regex pattern (matching is case in-senstive). Notice if you use symbols such as plus sign and others you would need to configure this using the RAW() syntax if configuring this as an endpoint uri. See more details at configuring endpoint uris
 | filter | filter |  | GenericFileFilter<T> | Pluggable filter as a org.apache.camel.component.file.GenericFileFilter class. Will skip files if filter returns false in its accept() method.
+| filterDirectory | filter |  | String | Filters the directory based on Simple language. For example to filter on current date you can use a simple date pattern such as $date:now:yyyMMdd
+| filterFile | filter |  | String | Filters the file based on Simple language. For example to filter on file size you can use $file:size 5000
 | idempotent | filter | false | Boolean | Option to use the Idempotent Consumer EIP pattern to let Camel skip already processed files. Will by default use a memory based LRUCache that holds 1000 entries. If noop=true then idempotent will be enabled as well to avoid consuming the same files over and over again.
 | idempotentKey | filter |  | String | To use a custom idempotent key. By default the absolute path of the file is used. You can use the File Language for example to use the file name and file size you can do: idempotentKey=$file:name-$file:size
 | idempotentRepository | filter |  | String> | A pluggable repository org.apache.camel.spi.IdempotentRepository which by default use MemoryMessageIdRepository if none is specified and idempotent is true.
@@ -224,6 +227,7 @@ The FTP component supports 100 endpoint options which are listed below:
 
 
 
+
 [Info]
 ====
 FTPS component default trust store