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/12/19 15:35:13 UTC

[1/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Repository: camel
Updated Branches:
  refs/heads/master ec735be05 -> 82dbe00cf


CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: 82dbe00cf9a939faf9c94714863e3ba275b9d1ad
Parents: 76218fb
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 16:12:31 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../mbean/ManagedReloadStrategyMBean.java       | 39 -----------
 .../camel/impl/FileWatcherReloadStrategy.java   | 43 ++++++++++--
 .../DefaultManagementLifecycleStrategy.java     |  4 --
 .../management/mbean/ManagedReloadStrategy.java | 72 --------------------
 .../org/apache/camel/spi/ReloadStrategy.java    |  6 +-
 .../camel/support/ReloadStrategySupport.java    | 15 ++--
 .../ManagedFileWatcherReloadStrategyTest.java   | 11 ++-
 examples/README.md                              |  1 +
 .../src/main/docs/camel-maven-plugin.adoc       | 24 +++++--
 9 files changed, 77 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
deleted file mode 100644
index dc13f27..0000000
--- a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.api.management.mbean;
-
-import org.apache.camel.api.management.ManagedAttribute;
-import org.apache.camel.api.management.ManagedOperation;
-
-public interface ManagedReloadStrategyMBean extends ManagedServiceMBean {
-
-    @ManagedOperation(description = "Force reload of CamelContext")
-    void forceReloadCamelContext();
-
-    @ManagedAttribute(description = "Strategy")
-    String getStrategy();
-
-    @ManagedAttribute(description = "Number of reloads succeeded")
-    int getReloadCounter();
-
-    @ManagedAttribute(description = "Number of reloads failed")
-    int getFailedCounter();
-
-    @ManagedOperation(description = "Reset counters")
-    void resetCounters();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
index 20573b7..6c2a321 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
@@ -27,13 +27,15 @@ import java.util.Locale;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
+import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
+
+import org.apache.camel.api.management.ManagedAttribute;
+import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.support.ReloadStrategySupport;
 import org.apache.camel.util.IOHelper;
 import org.apache.camel.util.ObjectHelper;
 
-import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
-import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
-
 /**
  * A file based {@link org.apache.camel.spi.ReloadStrategy} which watches a file folder
  * for modified files and reload on file changes.
@@ -43,10 +45,12 @@ import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
  * native file system changes and therefore the watch service is much slower than on
  * linux or windows systems.
  */
+@ManagedResource(description = "Managed FileWatcherReloadStrategy")
 public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
     private String folder;
     private ExecutorService executorService;
+    private WatchFileChangesTask task;
 
     public FileWatcherReloadStrategy() {
     }
@@ -55,11 +59,20 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
         setFolder(directory);
     }
 
-
     public void setFolder(String folder) {
         this.folder = folder;
     }
 
+    @ManagedAttribute(description = "Folder being watched")
+    public String getFolder() {
+        return folder;
+    }
+
+    @ManagedAttribute(description = "Whether the watcher is running")
+    public boolean isRunning() {
+        return task != null && task.isRunning();
+    }
+
     @Override
     protected void doStart() throws Exception {
         if (folder == null) {
@@ -70,14 +83,23 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
         File dir = new File(folder);
         if (dir.exists() && dir.isDirectory()) {
             log.info("Starting ReloadStrategy to watch directory: {}", dir);
+
+            // if its mac OSX then warn its slower
+            String os = ObjectHelper.getSystemProperty("os.name", "");
+            if (os.toLowerCase(Locale.US).startsWith("mac")) {
+                log.warn("On Mac OS X the JDK WatchService is slow and it may take up till 5 seconds to notice file changes");
+            }
+
             try {
                 Path path = dir.toPath();
                 WatchService watcher = path.getFileSystem().newWatchService();
                 // we cannot support deleting files as we don't know which routes that would be
                 path.register(watcher, ENTRY_CREATE, ENTRY_MODIFY);
 
+                task = new WatchFileChangesTask(watcher, path);
+
                 executorService = getCamelContext().getExecutorServiceManager().newSingleThreadExecutor(this, "FileWatcherReloadStrategy");
-                executorService.submit(new WatchFileChangesTask(watcher, path));
+                executorService.submit(task);
             } catch (IOException e) {
                 throw ObjectHelper.wrapRuntimeCamelException(e);
             }
@@ -100,17 +122,24 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
         private final WatchService watcher;
         private final Path folder;
+        private volatile boolean running;
 
         public WatchFileChangesTask(WatchService watcher, Path folder) {
             this.watcher = watcher;
             this.folder = folder;
         }
 
+        public boolean isRunning() {
+            return running;
+        }
+
         public void run() {
             log.debug("ReloadStrategy is starting watching folder: {}", folder);
 
             // allow to run while starting Camel
             while (isStarting() || isRunAllowed()) {
+                running = true;
+
                 WatchKey key;
                 try {
                     log.trace("ReloadStrategy is polling for file changes in directory: {}", folder);
@@ -132,7 +161,7 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
                             log.debug("Modified/Created XML file: {}", name);
                             try {
                                 FileInputStream fis = new FileInputStream(name);
-                                onReloadRoutes(getCamelContext(), name, fis);
+                                onReloadXml(getCamelContext(), name, fis);
                                 IOHelper.close(fis);
                             } catch (Exception e) {
                                 log.warn("Error reloading routes from file: " + name + " due " + e.getMessage() + ". This exception is ignored.", e);
@@ -148,6 +177,8 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
                 }
             }
 
+            running = false;
+
             log.info("ReloadStrategy is stopping watching folder: {}", folder);
         }
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
index 1df3a7e..34c8d5e 100644
--- a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
@@ -61,7 +61,6 @@ import org.apache.camel.management.mbean.ManagedEndpoint;
 import org.apache.camel.management.mbean.ManagedEndpointRegistry;
 import org.apache.camel.management.mbean.ManagedInflightRepository;
 import org.apache.camel.management.mbean.ManagedProducerCache;
-import org.apache.camel.management.mbean.ManagedReloadStrategy;
 import org.apache.camel.management.mbean.ManagedRestRegistry;
 import org.apache.camel.management.mbean.ManagedRoute;
 import org.apache.camel.management.mbean.ManagedRuntimeEndpointRegistry;
@@ -92,7 +91,6 @@ import org.apache.camel.spi.ManagementAware;
 import org.apache.camel.spi.ManagementNameStrategy;
 import org.apache.camel.spi.ManagementObjectStrategy;
 import org.apache.camel.spi.ManagementStrategy;
-import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.spi.RestRegistry;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.RuntimeEndpointRegistry;
@@ -489,8 +487,6 @@ public class DefaultManagementLifecycleStrategy extends ServiceSupport implement
             answer = new ManagedStreamCachingStrategy(context, (StreamCachingStrategy) service);
         } else if (service instanceof EventNotifier) {
             answer = getManagementObjectStrategy().getManagedObjectForEventNotifier(context, (EventNotifier) service);
-        } else if (service instanceof ReloadStrategy) {
-            answer = new ManagedReloadStrategy(context, (ReloadStrategy) service);
         } else if (service != null) {
             // fallback as generic service
             answer = getManagementObjectStrategy().getManagedObjectForService(context, service);

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
deleted file mode 100644
index 616ff5d..0000000
--- a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * 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.management.mbean;
-
-import org.apache.camel.CamelContext;
-import org.apache.camel.api.management.ManagedResource;
-import org.apache.camel.api.management.mbean.ManagedReloadStrategyMBean;
-import org.apache.camel.spi.ManagementStrategy;
-import org.apache.camel.spi.ReloadStrategy;
-
-/**
- * @version 
- */
-@ManagedResource(description = "Managed ReloadStrategy")
-public class ManagedReloadStrategy extends ManagedService implements ManagedReloadStrategyMBean {
-
-    private final CamelContext camelContext;
-    private final ReloadStrategy reloadStrategy;
-
-    public ManagedReloadStrategy(CamelContext camelContext, ReloadStrategy reloadStrategy) {
-        super(camelContext, reloadStrategy);
-        this.camelContext = camelContext;
-        this.reloadStrategy = reloadStrategy;
-    }
-
-    public CamelContext getCamelContext() {
-        return camelContext;
-    }
-
-    public ReloadStrategy getReloadStrategy() {
-        return reloadStrategy;
-    }
-
-    @Override
-    public void forceReloadCamelContext() {
-        reloadStrategy.onReloadCamelContext(getContext());
-    }
-
-    @Override
-    public String getStrategy() {
-        return reloadStrategy.getClass().getSimpleName();
-    }
-
-    @Override
-    public int getReloadCounter() {
-        return reloadStrategy.getReloadCounter();
-    }
-
-    @Override
-    public int getFailedCounter() {
-        return reloadStrategy.getFailedCounter();
-    }
-
-    @Override
-    public void resetCounters() {
-        reloadStrategy.resetCounters();
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
index 0385301..9469adb 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
@@ -28,8 +28,6 @@ import org.apache.camel.StaticService;
  */
 public interface ReloadStrategy extends Service, StaticService, CamelContextAware {
 
-    // TODO: naming of this SPI?
-
     /**
      * A reload is triggered and the {@link CamelContext} is expected to do a re-start
      *
@@ -38,13 +36,13 @@ public interface ReloadStrategy extends Service, StaticService, CamelContextAwar
     void onReloadCamelContext(CamelContext camelContext);
 
     /**
-     * A reload is triggered with a resource of some sort has been changed, such as a xml file with Camel routes.
+     * A reload is triggered when a XML resource is changed which contains Camel routes or rest services
      *
      * @param camelContext  the running CamelContext
      * @param name          name of resource such as a file name (can be null)
      * @param resource      the changed resource
      */
-    void onReloadRoutes(CamelContext camelContext, String name, InputStream resource);
+    void onReloadXml(CamelContext camelContext, String name, InputStream resource);
 
     /**
      * Number of reloads succeeded.

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
index a2b1ec0..6ae317e 100644
--- a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
@@ -23,6 +23,8 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.api.management.ManagedAttribute;
+import org.apache.camel.api.management.ManagedOperation;
 import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.RoutesDefinition;
 import org.apache.camel.spi.ReloadStrategy;
@@ -67,8 +69,8 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
     }
 
     @Override
-    public void onReloadRoutes(CamelContext camelContext, String name, InputStream resource) {
-        log.debug("Reloading CamelContext: {} routes from resource: {}", camelContext.getName(), name);
+    public void onReloadXml(CamelContext camelContext, String name, InputStream resource) {
+        log.debug("Reloading CamelContext: {} from XML resource: {}", camelContext.getName(), name);
 
         Document dom = camelContext.getTypeConverter().tryConvertTo(Document.class, resource);
         if (dom == null) {
@@ -94,25 +96,24 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
                     failed++;
                     throw ObjectHelper.wrapRuntimeCamelException(e);
                 }
-
             }
         }
 
-        log.info("Reloaded CamelContext: {} routes from resource: {}", camelContext.getName(), name);
+        log.info("Reloaded CamelContext: {} from XML resource: {}", camelContext.getName(), name);
         succeeded++;
     }
 
-    @Override
+    @ManagedAttribute(description = "Number of reloads succeeded")
     public int getReloadCounter() {
         return succeeded;
     }
 
-    @Override
+    @ManagedAttribute(description = "Number of reloads failed")
     public int getFailedCounter() {
         return failed;
     }
 
-    @Override
+    @ManagedOperation(description = "Reset counters")
     public void resetCounters() {
         succeeded = 0;
         failed = 0;

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java b/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
index 76e7aa5..136ccd04 100644
--- a/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
+++ b/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
@@ -38,8 +38,11 @@ public class ManagedFileWatcherReloadStrategyTest extends ManagementTestSupport
         ObjectName on = ObjectName.getInstance("org.apache.camel:context=camel-1,type=services,name=FileWatcherReloadStrategy");
         assertTrue(mbeanServer.isRegistered(on));
 
-        String strategy = (String) mbeanServer.getAttribute(on, "Strategy");
-        assertEquals("FileWatcherReloadStrategy", strategy);
+        String folder = (String) mbeanServer.getAttribute(on, "Folder");
+        assertEquals("target/dummy", folder);
+
+        Boolean running = (Boolean) mbeanServer.getAttribute(on, "Running");
+        assertTrue(running);
 
         Integer reload = (Integer) mbeanServer.getAttribute(on, "ReloadCounter");
         assertEquals(0, reload.intValue());
@@ -53,6 +56,10 @@ public class ManagedFileWatcherReloadStrategyTest extends ManagementTestSupport
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
+                // directory must exists for the watcher to be able to run
+                deleteDirectory("target/dummy");
+                createDirectory("target/dummy");
+
                 // add reload strategy
                 context.setReloadStrategy(new FileWatcherReloadStrategy("target/dummy"));
 

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/examples/README.md
----------------------------------------------------------------------
diff --git a/examples/README.md b/examples/README.md
index 1e3d97d..53815db 100644
--- a/examples/README.md
+++ b/examples/README.md
@@ -19,6 +19,7 @@ All examples have been sort by type/category
 * [camel-example-console](camel-example-console/README.md)
 * [camel-example-jmx](camel-example-jmx/README.md)
 * [camel-example-ftp](camel-example-ftp/README.md)
+* [camel-example-reload](camel-example-reload/Readme.md)
 * [camel-example-ssh](camel-example-ssh/README.md)
 * [camel-example-ssh-security](camel-example-ssh-security/README.md)
 

http://git-wip-us.apache.org/repos/asf/camel/blob/82dbe00c/tooling/maven/camel-maven-plugin/src/main/docs/camel-maven-plugin.adoc
----------------------------------------------------------------------
diff --git a/tooling/maven/camel-maven-plugin/src/main/docs/camel-maven-plugin.adoc b/tooling/maven/camel-maven-plugin/src/main/docs/camel-maven-plugin.adoc
index a9ceb21..01787e4 100644
--- a/tooling/maven/camel-maven-plugin/src/main/docs/camel-maven-plugin.adoc
+++ b/tooling/maven/camel-maven-plugin/src/main/docs/camel-maven-plugin.adoc
@@ -21,7 +21,7 @@ then boot up a Spring ApplicationContext using the XML configuration files on th
 
 If you want to boot up your Camel routes a little faster, you could try the `camel:embedded` instead.
 
-== Running OSGi Blueprint
+=== Running OSGi Blueprint
 
 The `camel:run` plugin also supports running a Blueprint application, and by default it scans for OSGi blueprint files in
 `OSGI-INF/blueprint/*.xml`
@@ -67,8 +67,7 @@ loading the persistence properties file.
 
 The `configAdminFileName` is the file name which will be used to load the configuration admin service properties file.
 
-
-== Running CDI
+=== Running CDI
 
 The `camel:run` plugin also supports running a CDI application
 
@@ -80,7 +79,7 @@ From the source of Camel you can run a CDI example via
     cd examples/camel-example-cdi
     mvn compile camel:run
 
-== Logging the classpath
+=== Logging the classpath
 
 You can configure whether the classpath should be logged when `camel:run` executes.
 You can enable this in the configuration using:
@@ -93,6 +92,23 @@ You can enable this in the configuration using:
       </configuration>
     </plugin>
 
+=== Using live reload of XML files
+
+You can configure the plugin to scan for XML file changes and trigger a reload of the Camel routes which are contained in those XML files.
+
+    <plugin>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-maven-plugin</artifactId>
+      <configuration>
+        <fileWatcherDirectory>src/main/resources/META-INF/spring</fileWatcherDirectory>
+      </configuration>
+    </plugin>
+
+Then the plugin watches this directory. This allows you to edit the source code from your editor and save the file, and have
+    the running Camel application pickup those changes.
+
+Notice its only changes of Camel routes, eg `<routes>`, or `<route>` which is supported.
+You cannot change Spring or OSGi Blueprint `<bean>` elements.
 
 == camel:validate
 


[3/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: 417c57d7feeccafd15403f4ad1c1969bf6e1f096
Parents: d9fb32e
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 14:00:50 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../mbean/ManagedReloadStrategyMBean.java       | 32 +++++++++++++
 .../DefaultManagementLifecycleStrategy.java     |  4 ++
 .../management/mbean/ManagedReloadStrategy.java | 50 ++++++++++++++++++++
 .../org/apache/camel/spi/ReloadStrategy.java    | 11 ++++-
 .../camel/support/ReloadStrategySupport.java    | 23 +++++++++
 5 files changed, 119 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/417c57d7/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
new file mode 100644
index 0000000..80065ce
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
@@ -0,0 +1,32 @@
+/**
+ * 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.api.management.mbean;
+
+import org.apache.camel.api.management.ManagedAttribute;
+
+public interface ManagedReloadStrategyMBean extends ManagedServiceMBean {
+
+    @ManagedAttribute(description = "Strategy")
+    String getStrategy();
+
+    @ManagedAttribute(description = "Number of reloads succeeded")
+    int getReloadCounter();
+
+    @ManagedAttribute(description = "Number of reloads failed")
+    int getFailedCounter();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/417c57d7/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
index 34c8d5e..1df3a7e 100644
--- a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
@@ -61,6 +61,7 @@ import org.apache.camel.management.mbean.ManagedEndpoint;
 import org.apache.camel.management.mbean.ManagedEndpointRegistry;
 import org.apache.camel.management.mbean.ManagedInflightRepository;
 import org.apache.camel.management.mbean.ManagedProducerCache;
+import org.apache.camel.management.mbean.ManagedReloadStrategy;
 import org.apache.camel.management.mbean.ManagedRestRegistry;
 import org.apache.camel.management.mbean.ManagedRoute;
 import org.apache.camel.management.mbean.ManagedRuntimeEndpointRegistry;
@@ -91,6 +92,7 @@ import org.apache.camel.spi.ManagementAware;
 import org.apache.camel.spi.ManagementNameStrategy;
 import org.apache.camel.spi.ManagementObjectStrategy;
 import org.apache.camel.spi.ManagementStrategy;
+import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.spi.RestRegistry;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.spi.RuntimeEndpointRegistry;
@@ -487,6 +489,8 @@ public class DefaultManagementLifecycleStrategy extends ServiceSupport implement
             answer = new ManagedStreamCachingStrategy(context, (StreamCachingStrategy) service);
         } else if (service instanceof EventNotifier) {
             answer = getManagementObjectStrategy().getManagedObjectForEventNotifier(context, (EventNotifier) service);
+        } else if (service instanceof ReloadStrategy) {
+            answer = new ManagedReloadStrategy(context, (ReloadStrategy) service);
         } else if (service != null) {
             // fallback as generic service
             answer = getManagementObjectStrategy().getManagedObjectForService(context, service);

http://git-wip-us.apache.org/repos/asf/camel/blob/417c57d7/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
new file mode 100644
index 0000000..3635c7c
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
@@ -0,0 +1,50 @@
+/**
+ * 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.management.mbean;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.api.management.ManagedResource;
+import org.apache.camel.api.management.mbean.ManagedReloadStrategyMBean;
+import org.apache.camel.spi.ReloadStrategy;
+
+/**
+ * @version 
+ */
+@ManagedResource(description = "Managed ReloadStrategy")
+public class ManagedReloadStrategy extends ManagedService implements ManagedReloadStrategyMBean {
+    private final ReloadStrategy reloadStrategy;
+
+    public ManagedReloadStrategy(CamelContext context, ReloadStrategy reloadStrategy) {
+        super(context, reloadStrategy);
+        this.reloadStrategy = reloadStrategy;
+    }
+
+    @Override
+    public String getStrategy() {
+        return reloadStrategy.getClass().getSimpleName();
+    }
+
+    @Override
+    public int getReloadCounter() {
+        return reloadStrategy.getReloadCounter();
+    }
+
+    @Override
+    public int getFailedCounter() {
+        return reloadStrategy.getFailedCounter();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/417c57d7/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
index 7f30552..3c74dac 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
@@ -29,7 +29,6 @@ import org.apache.camel.StaticService;
 public interface ReloadStrategy extends Service, StaticService, CamelContextAware {
 
     // TODO: naming of this SPI?
-    // TODO: Add JMX mbean
 
     /**
      * A reload is triggered and the {@link CamelContext} is expected to do a re-start
@@ -46,4 +45,14 @@ public interface ReloadStrategy extends Service, StaticService, CamelContextAwar
      * @param resource      the changed resource
      */
     void onReloadRoutes(CamelContext camelContext, String name, InputStream resource);
+
+    /**
+     * Number of reloads succeeded.
+     */
+    int getReloadCounter();
+
+    /**
+     * Number of reloads failed.
+     */
+    int getFailedCounter();
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/417c57d7/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
index 355f19d..e0957d7 100644
--- a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
@@ -33,6 +33,9 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
     protected final Logger log = LoggerFactory.getLogger(getClass());
     private CamelContext camelContext;
 
+    private int succeeded;
+    private int failed;
+
     @Override
     public CamelContext getCamelContext() {
         return camelContext;
@@ -50,21 +53,41 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
             ServiceHelper.stopService(camelContext);
             ServiceHelper.startService(camelContext);
         } catch (Exception e) {
+            failed++;
             throw ObjectHelper.wrapRuntimeCamelException(e);
         }
         log.info("Reloaded CamelContext: {}", camelContext.getName());
+
+        succeeded++;
     }
 
     @Override
     public void onReloadRoutes(CamelContext camelContext, String name, InputStream resource) {
+
+        // load the stream in as DOM and find out if its <routes> <route> or <camelContext>
+        // and if its <blueprint> <beans> etc and then find inside the <camelContext> and grab what we support re-loading
+
         log.debug("Reloading CamelContext: {} routes from resource: {}", camelContext.getName(), name);
         // assume the resource is XML routes
         try {
             RoutesDefinition routes = camelContext.loadRoutesDefinition(resource);
             camelContext.addRouteDefinitions(routes.getRoutes());
         } catch (Exception e) {
+            failed++;
             throw ObjectHelper.wrapRuntimeCamelException(e);
         }
         log.info("Reloaded CamelContext: {} routes from resource: {}", camelContext.getName(), name);
+
+        succeeded++;
+    }
+
+    @Override
+    public int getReloadCounter() {
+        return succeeded;
+    }
+
+    @Override
+    public int getFailedCounter() {
+        return failed;
     }
 }


[4/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: 21315271a271b6e2cbf21a87f10e334e7f2a771c
Parents: 417c57d
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 14:34:36 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../camel/impl/FileWatcherReloadStrategy.java   |   4 +
 .../java/org/apache/camel/main/MainSupport.java |  28 ++-
 examples/camel-example-reload/Readme.md         |  32 +++
 examples/camel-example-reload/pom.xml           |  82 ++++++++
 .../camel/example/reload/CamelReloadMain.java   |  39 ++++
 .../src/main/resources/META-INF/LICENSE.txt     | 203 +++++++++++++++++++
 .../src/main/resources/META-INF/NOTICE.txt      |  11 +
 .../resources/META-INF/spring/camel-context.xml |  36 ++++
 .../src/main/resources/log4j2.properties        |  23 +++
 examples/pom.xml                                |   1 +
 .../java/org/apache/camel/maven/RunMojo.java    |  12 ++
 11 files changed, 468 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
index f7bb58a..3fc3dd1 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
@@ -49,6 +49,10 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
     public FileWatcherReloadStrategy() {
     }
 
+    public FileWatcherReloadStrategy(String directory) {
+        setFolder(directory);
+    }
+
     public Path getFolder() {
         return folder;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
index dc7bc97..f84f52e 100644
--- a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
+++ b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
@@ -32,7 +32,7 @@ import org.apache.camel.CamelContext;
 import org.apache.camel.ProducerTemplate;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.DefaultModelJAXBContextFactory;
-import org.apache.camel.model.ModelCamelContext;
+import org.apache.camel.impl.FileWatcherReloadStrategy;
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.spi.ModelJAXBContextFactory;
 import org.apache.camel.spi.ReloadStrategy;
@@ -60,6 +60,7 @@ public abstract class MainSupport extends ServiceSupport {
     protected boolean trace;
     protected List<RouteBuilder> routeBuilders = new ArrayList<RouteBuilder>();
     protected String routeBuilderClasses;
+    protected String fileWatchDirectory;
     protected final List<CamelContext> camelContexts = new ArrayList<CamelContext>();
     protected ProducerTemplate camelTemplate;
     protected boolean hangupInterceptorEnabled = true;
@@ -127,6 +128,14 @@ public abstract class MainSupport extends ServiceSupport {
                 setDurationHitExitCode(Integer.parseInt(parameter));
             }
         });
+        addOption(new ParameterOption("watch", "fileWatch",
+                "Sets a directory to watch for file changes to trigger reloading routes on-the-fly",
+                "fileWatch") {
+            @Override
+            protected void doProcess(String arg, String parameter, LinkedList<String> remainingArgs) {
+                setFileWatchDirectory(parameter);
+            }
+        });
     }
 
     /**
@@ -341,11 +350,18 @@ public abstract class MainSupport extends ServiceSupport {
         return exitCode.get();
     }
 
-
     public void setRouteBuilderClasses(String builders) {
         this.routeBuilderClasses = builders;
     }
 
+    public String getFileWatchDirectory() {
+        return fileWatchDirectory;
+    }
+
+    public void setFileWatchDirectory(String fileWatchDirectory) {
+        this.fileWatchDirectory = fileWatchDirectory;
+    }
+
     public String getRouteBuilderClasses() {
         return routeBuilderClasses;
     }
@@ -423,7 +439,7 @@ public abstract class MainSupport extends ServiceSupport {
     public List<RouteDefinition> getRouteDefinitions() {
         List<RouteDefinition> answer = new ArrayList<RouteDefinition>();
         for (CamelContext camelContext : camelContexts) {
-            answer.addAll(((ModelCamelContext)camelContext).getRouteDefinitions());
+            answer.addAll(camelContext.getRouteDefinitions());
         }
         return answer;
     }
@@ -469,6 +485,12 @@ public abstract class MainSupport extends ServiceSupport {
         if (trace) {
             camelContext.setTracing(true);
         }
+        if (fileWatchDirectory != null) {
+            ReloadStrategy reload = new FileWatcherReloadStrategy(fileWatchDirectory);
+            camelContext.setReloadStrategy(reload);
+            // ensure reload is added as service and started
+            camelContext.addService(reload);
+        }
         // try to load the route builders from the routeBuilderClasses
         loadRouteBuilders(camelContext);
         for (RouteBuilder routeBuilder : routeBuilders) {

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/Readme.md
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/Readme.md b/examples/camel-example-reload/Readme.md
new file mode 100644
index 0000000..60e0903
--- /dev/null
+++ b/examples/camel-example-reload/Readme.md
@@ -0,0 +1,32 @@
+Camel Example Reload
+====================
+
+This example runs a standalone Camel Spring application where the routes are defined in
+the Spring XML located in `src/main/resources/META-INF/spring/camel-context.xml`.
+
+At runtime you can modify this file in the source code and then Camel will automatic reload
+the route on-the-fly.
+
+### How to try
+
+You need to build the example first with
+
+    mvn compile
+    
+Then you can run it from Maven
+    
+    mvn camel:run
+    
+Or you can run it from your Java IDE editor by running the following Main class
+    
+    org.apache.camel.example.reload.CamelReloadMain
+    
+### Reload changes
+    
+When the example is running then try to modify the XML file such as changing the message to be `Bye World`
+    
+      <transform>
+        <constant>Bye World</constant>
+      </transform>
+
+You modify the source file directory in the `src/main/resources/META-INF/spring/` directory

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/pom.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/pom.xml b/examples/camel-example-reload/pom.xml
new file mode 100644
index 0000000..4c951fc
--- /dev/null
+++ b/examples/camel-example-reload/pom.xml
@@ -0,0 +1,82 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>examples</artifactId>
+    <version>2.19.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>camel-example-reload</artifactId>
+  <packaging>jar</packaging>
+  <name>Camel :: Example :: Reload</name>
+  <description>An example that live reloads routes when the XML file is updated</description>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-spring</artifactId>
+    </dependency>
+
+    <!-- logging -->
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-core</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-slf4j-impl</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <plugins>
+
+      <!-- Allows the routes to be run via 'mvn camel:run' -->
+      <plugin>
+        <groupId>org.apache.camel</groupId>
+        <artifactId>camel-maven-plugin</artifactId>
+        <version>${project.version}</version>
+        <configuration>
+          <!-- turn on reload when the XML file is updated in the source code -->
+          <fileWatcherDirectory>src/main/resources/META-INF/spring</fileWatcherDirectory>
+        </configuration>
+      </plugin>
+
+    </plugins>
+
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/src/main/java/org/apache/camel/example/reload/CamelReloadMain.java
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/java/org/apache/camel/example/reload/CamelReloadMain.java b/examples/camel-example-reload/src/main/java/org/apache/camel/example/reload/CamelReloadMain.java
new file mode 100644
index 0000000..bf44247
--- /dev/null
+++ b/examples/camel-example-reload/src/main/java/org/apache/camel/example/reload/CamelReloadMain.java
@@ -0,0 +1,39 @@
+/**
+ * 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.example.reload;
+
+import org.apache.camel.spring.Main;
+
+/**
+ * A main class to run the example from your editor.
+ */
+public final class CamelReloadMain {
+
+    private CamelReloadMain() {
+    }
+
+    public static void main(String[] args) throws Exception {
+        // Main makes it easier to run a Spring application
+        Main main = new Main();
+        // configure the location of the Spring XML file
+        main.setApplicationContextUri("META-INF/spring/camel-context.xml");
+        // turn on reload when the XML file is updated in the source code
+        main.setFileWatchDirectory("src/main/resources/META-INF/spring");
+        // run and block until Camel is stopped (or JVM terminated)
+        main.run();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/src/main/resources/META-INF/LICENSE.txt
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/resources/META-INF/LICENSE.txt b/examples/camel-example-reload/src/main/resources/META-INF/LICENSE.txt
new file mode 100644
index 0000000..6b0b127
--- /dev/null
+++ b/examples/camel-example-reload/src/main/resources/META-INF/LICENSE.txt
@@ -0,0 +1,203 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/src/main/resources/META-INF/NOTICE.txt
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/resources/META-INF/NOTICE.txt b/examples/camel-example-reload/src/main/resources/META-INF/NOTICE.txt
new file mode 100644
index 0000000..2e215bf
--- /dev/null
+++ b/examples/camel-example-reload/src/main/resources/META-INF/NOTICE.txt
@@ -0,0 +1,11 @@
+   =========================================================================
+   ==  NOTICE file corresponding to the section 4 d of                    ==
+   ==  the Apache License, Version 2.0,                                   ==
+   ==  in this case for the Apache Camel distribution.                    ==
+   =========================================================================
+
+   This product includes software developed by
+   The Apache Software Foundation (http://www.apache.org/).
+
+   Please read the different LICENSE files present in the licenses directory of
+   this distribution.

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml b/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
new file mode 100644
index 0000000..ec81cc1
--- /dev/null
+++ b/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+       http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd">
+
+  <camelContext xmlns="http://camel.apache.org/schema/spring">
+    <route>
+      <from uri="timer:foo"/>
+      <!-- try to change me and save this file -->
+      <transform>
+        <constant>Hello World</constant>
+      </transform>
+      <to uri="log:out"/>
+    </route>
+  </camelContext>
+
+</beans>

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/camel-example-reload/src/main/resources/log4j2.properties
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/resources/log4j2.properties b/examples/camel-example-reload/src/main/resources/log4j2.properties
new file mode 100644
index 0000000..93d8e72
--- /dev/null
+++ b/examples/camel-example-reload/src/main/resources/log4j2.properties
@@ -0,0 +1,23 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+
+appender.out.type = Console
+appender.out.name = out
+appender.out.layout.type = PatternLayout
+appender.out.layout.pattern = [%30.30t] %-30.30c{1} %-5p %m%n
+rootLogger.level = INFO
+rootLogger.appenderRef.out.ref = out

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 0cebc70..cda2ed9 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -69,6 +69,7 @@
     <module>camel-example-netty-http</module>
     <module>camel-example-osgi-rmi</module>
     <module>camel-example-pojo-messaging</module>
+    <module>camel-example-reload</module>
     <module>camel-example-reportincident</module>
     <module>camel-example-reportincident-wssecurity</module>
     <module>camel-example-restlet-jdbc</module>

http://git-wip-us.apache.org/repos/asf/camel/blob/21315271/tooling/maven/camel-maven-plugin/src/main/java/org/apache/camel/maven/RunMojo.java
----------------------------------------------------------------------
diff --git a/tooling/maven/camel-maven-plugin/src/main/java/org/apache/camel/maven/RunMojo.java b/tooling/maven/camel-maven-plugin/src/main/java/org/apache/camel/maven/RunMojo.java
index 8fcc80c..f7170b6 100644
--- a/tooling/maven/camel-maven-plugin/src/main/java/org/apache/camel/maven/RunMojo.java
+++ b/tooling/maven/camel-maven-plugin/src/main/java/org/apache/camel/maven/RunMojo.java
@@ -217,6 +217,14 @@ public class RunMojo extends AbstractExecMojo {
     private String configAdminFileName;
 
     /**
+     * To watch the directory for file changes which triggers
+     * a live reload of the Camel routes on-the-fly.
+     *
+     * @parameter property="camel.fileWatcherDirectory"
+     */
+    private String fileWatcherDirectory;
+
+    /**
      * The class arguments.
      *
      * @parameter property="camel.arguments"
@@ -393,6 +401,10 @@ public class RunMojo extends AbstractExecMojo {
         if (trace) {
             args.add("-t");
         }
+        if (fileWatcherDirectory != null) {
+            args.add("-watch");
+            args.add(fileWatcherDirectory);
+        }
 
         if (applicationContextUri != null) {
             args.add("-ac");


[2/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: b868b5f7a3bf839dc6e96834007cb5fbdaea6a05
Parents: 4df1650
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 15:32:45 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../mbean/ManagedReloadStrategyMBean.java       |  3 +++
 .../camel/management/mbean/ManagedConsumer.java |  5 -----
 .../management/mbean/ManagedReloadStrategy.java | 21 ++++++++++++++++++--
 .../mbean/ManagedStreamCachingStrategy.java     |  4 ----
 .../org/apache/camel/model/ModelHelper.java     |  4 +++-
 .../org/apache/camel/spi/ReloadStrategy.java    |  5 +++++
 .../camel/support/ReloadStrategySupport.java    |  6 ++++++
 7 files changed, 36 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
index 3154557..dc13f27 100644
--- a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
+++ b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
@@ -33,4 +33,7 @@ public interface ManagedReloadStrategyMBean extends ManagedServiceMBean {
     @ManagedAttribute(description = "Number of reloads failed")
     int getFailedCounter();
 
+    @ManagedOperation(description = "Reset counters")
+    void resetCounters();
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedConsumer.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedConsumer.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedConsumer.java
index 1506365..a940286 100644
--- a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedConsumer.java
+++ b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedConsumer.java
@@ -34,11 +34,6 @@ public class ManagedConsumer extends ManagedService implements ManagedConsumerMB
         this.consumer = consumer;
     }
 
-    @Override
-    public void init(ManagementStrategy strategy) {
-        // noop
-    }
-
     public Consumer getConsumer() {
         return consumer;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
index 9660a30..616ff5d 100644
--- a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
@@ -19,6 +19,7 @@ package org.apache.camel.management.mbean;
 import org.apache.camel.CamelContext;
 import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.api.management.mbean.ManagedReloadStrategyMBean;
+import org.apache.camel.spi.ManagementStrategy;
 import org.apache.camel.spi.ReloadStrategy;
 
 /**
@@ -26,13 +27,24 @@ import org.apache.camel.spi.ReloadStrategy;
  */
 @ManagedResource(description = "Managed ReloadStrategy")
 public class ManagedReloadStrategy extends ManagedService implements ManagedReloadStrategyMBean {
+
+    private final CamelContext camelContext;
     private final ReloadStrategy reloadStrategy;
 
-    public ManagedReloadStrategy(CamelContext context, ReloadStrategy reloadStrategy) {
-        super(context, reloadStrategy);
+    public ManagedReloadStrategy(CamelContext camelContext, ReloadStrategy reloadStrategy) {
+        super(camelContext, reloadStrategy);
+        this.camelContext = camelContext;
         this.reloadStrategy = reloadStrategy;
     }
 
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    public ReloadStrategy getReloadStrategy() {
+        return reloadStrategy;
+    }
+
     @Override
     public void forceReloadCamelContext() {
         reloadStrategy.onReloadCamelContext(getContext());
@@ -52,4 +64,9 @@ public class ManagedReloadStrategy extends ManagedService implements ManagedRelo
     public int getFailedCounter() {
         return reloadStrategy.getFailedCounter();
     }
+
+    @Override
+    public void resetCounters() {
+        reloadStrategy.resetCounters();
+    }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedStreamCachingStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedStreamCachingStrategy.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedStreamCachingStrategy.java
index 284ac52..99d88e9 100644
--- a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedStreamCachingStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedStreamCachingStrategy.java
@@ -34,10 +34,6 @@ public class ManagedStreamCachingStrategy extends ManagedService implements Mana
         this.streamCachingStrategy = streamCachingStrategy;
     }
 
-    public void init(ManagementStrategy strategy) {
-        // do nothing
-    }
-
     public CamelContext getCamelContext() {
         return camelContext;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
index 3a41d1a..b681b38 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
@@ -160,7 +160,9 @@ public final class ModelHelper {
         JAXBContext jaxbContext = getJAXBContext(context);
 
         Map<String, String> namespaces = new LinkedHashMap<>();
-        extractNamespaces(node.getOwnerDocument(), namespaces);
+
+        Document dom = node instanceof Document ? (Document) node : node.getOwnerDocument();
+        extractNamespaces(dom, namespaces);
 
         Binder<Node> binder = jaxbContext.createBinder();
         Object result = binder.unmarshal(node);

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
index 3c74dac..0385301 100644
--- a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
@@ -55,4 +55,9 @@ public interface ReloadStrategy extends Service, StaticService, CamelContextAwar
      * Number of reloads failed.
      */
     int getFailedCounter();
+
+    /**
+     * Reset the counters.
+     */
+    void resetCounters();
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/b868b5f7/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
index aa775f5..a2b1ec0 100644
--- a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
@@ -111,4 +111,10 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
     public int getFailedCounter() {
         return failed;
     }
+
+    @Override
+    public void resetCounters() {
+        succeeded = 0;
+        failed = 0;
+    }
 }


[8/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: 4df165016ce93590f3856af458ab56f68c1c0981
Parents: fbbf829
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 15:20:45 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../mbean/ManagedReloadStrategyMBean.java        |  4 ++++
 .../camel/impl/FileWatcherReloadStrategy.java    | 17 ++++++++++++++---
 .../java/org/apache/camel/main/MainSupport.java  | 19 +++++++++++++++++++
 .../management/mbean/ManagedReloadStrategy.java  |  5 +++++
 .../java/org/apache/camel/model/ModelHelper.java |  9 +++++----
 .../camel/support/ReloadStrategySupport.java     |  7 ++++---
 6 files changed, 51 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
index 80065ce..3154557 100644
--- a/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
+++ b/camel-core/src/main/java/org/apache/camel/api/management/mbean/ManagedReloadStrategyMBean.java
@@ -17,9 +17,13 @@
 package org.apache.camel.api.management.mbean;
 
 import org.apache.camel.api.management.ManagedAttribute;
+import org.apache.camel.api.management.ManagedOperation;
 
 public interface ManagedReloadStrategyMBean extends ManagedServiceMBean {
 
+    @ManagedOperation(description = "Force reload of CamelContext")
+    void forceReloadCamelContext();
+
     @ManagedAttribute(description = "Strategy")
     String getStrategy();
 

http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
index 3fc3dd1..e80f048 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
@@ -37,11 +37,14 @@ import org.apache.camel.util.ObjectHelper;
 /**
  * A file based {@link org.apache.camel.spi.ReloadStrategy} which watches a file folder
  * for modified files and reload on file changes.
+ * <p/>
+ * This implementation uses the JDK {@link WatchService} to watch for when files are
+ * created or modified. Mac OS X users should be noted the osx JDK does not support
+ * native file system changes and therefore the watch service is much slower than on
+ * linux or windows systems.
  */
 public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
-    // TODO: support multiple folders
-
     private Path folder;
     private WatchService watcher;
     private ExecutorService executorService;
@@ -63,6 +66,11 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
     @Override
     protected void doStart() throws Exception {
+        if (folder == null) {
+            // no folder configured
+            return;
+        }
+
         log.info("Starting ReloadStrategy to watch directory: {}", folder);
         try {
             this.watcher = folder.getFileSystem().newWatchService();
@@ -79,7 +87,10 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
     @Override
     protected void doStop() throws Exception {
-        getCamelContext().getExecutorServiceManager().shutdownGraceful(executorService);
+        if (executorService != null) {
+            getCamelContext().getExecutorServiceManager().shutdownGraceful(executorService);
+            executorService = null;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
index f84f52e..cb77814 100644
--- a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
+++ b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
@@ -490,7 +490,26 @@ public abstract class MainSupport extends ServiceSupport {
             camelContext.setReloadStrategy(reload);
             // ensure reload is added as service and started
             camelContext.addService(reload);
+            // and ensure its register in JMX (which requires manually to be added because CamelContext is already started)
+            Object managedObject = camelContext.getManagementStrategy().getManagementObjectStrategy().getManagedObjectForService(camelContext, reload);
+            if (managedObject == null) {
+                // service should not be managed
+                return;
+            }
+
+            // skip already managed services, for example if a route has been restarted
+            if (camelContext.getManagementStrategy().isManaged(managedObject, null)) {
+                LOG.trace("The service is already managed: {}", reload);
+                return;
+            }
+
+            try {
+                camelContext.getManagementStrategy().manageObject(managedObject);
+            } catch (Exception e) {
+                LOG.warn("Could not register service: " + reload + " as Service MBean.", e);
+            }
         }
+
         // try to load the route builders from the routeBuilderClasses
         loadRouteBuilders(camelContext);
         for (RouteBuilder routeBuilder : routeBuilders) {

http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
index 3635c7c..9660a30 100644
--- a/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedReloadStrategy.java
@@ -34,6 +34,11 @@ public class ManagedReloadStrategy extends ManagedService implements ManagedRelo
     }
 
     @Override
+    public void forceReloadCamelContext() {
+        reloadStrategy.onReloadCamelContext(getContext());
+    }
+
+    @Override
     public String getStrategy() {
         return reloadStrategy.getClass().getSimpleName();
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
index 871bdad..3a41d1a 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
@@ -30,6 +30,11 @@ import javax.xml.bind.Marshaller;
 import javax.xml.transform.OutputKeys;
 import javax.xml.transform.TransformerException;
 
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+
 import org.apache.camel.CamelContext;
 import org.apache.camel.Expression;
 import org.apache.camel.NamedNode;
@@ -39,10 +44,6 @@ import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.spi.NamespaceAware;
 import org.apache.camel.spi.TypeConverterRegistry;
 import org.apache.camel.util.ObjectHelper;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
 
 import static org.apache.camel.model.ProcessorDefinitionHelper.filterTypeInOutputs;
 

http://git-wip-us.apache.org/repos/asf/camel/blob/4df16501/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
index 7dbc14b..aa775f5 100644
--- a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
@@ -18,6 +18,10 @@ package org.apache.camel.support;
 
 import java.io.InputStream;
 
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
 import org.apache.camel.CamelContext;
 import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.RoutesDefinition;
@@ -26,9 +30,6 @@ import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 /**
  * Base class for implementing custom {@link ReloadStrategy} SPI plugins.


[6/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: fbbf82997a8a40e706b539455daa4549e8eda92e
Parents: 2131527
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 14:51:37 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../org/apache/camel/model/ModelHelper.java     | 27 ++++++++-----
 .../camel/support/ReloadStrategySupport.java    | 42 +++++++++++++++-----
 .../resources/META-INF/spring/camel-context.xml |  4 +-
 3 files changed, 51 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/fbbf8299/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
index db40f63..871bdad 100644
--- a/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
+++ b/camel-core/src/main/java/org/apache/camel/model/ModelHelper.java
@@ -30,11 +30,6 @@ import javax.xml.bind.Marshaller;
 import javax.xml.transform.OutputKeys;
 import javax.xml.transform.TransformerException;
 
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NamedNodeMap;
-import org.w3c.dom.Node;
-
 import org.apache.camel.CamelContext;
 import org.apache.camel.Expression;
 import org.apache.camel.NamedNode;
@@ -44,6 +39,10 @@ import org.apache.camel.model.language.ExpressionDefinition;
 import org.apache.camel.spi.NamespaceAware;
 import org.apache.camel.spi.TypeConverterRegistry;
 import org.apache.camel.util.ObjectHelper;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
 
 import static org.apache.camel.model.ProcessorDefinitionHelper.filterTypeInOutputs;
 
@@ -144,16 +143,26 @@ public final class ModelHelper {
      * @throws Exception is thrown if an error is encountered unmarshalling from xml to model
      */
     public static RoutesDefinition loadRoutesDefinition(CamelContext context, InputStream inputStream) throws Exception {
-        JAXBContext jaxbContext = getJAXBContext(context);
-
         XmlConverter xmlConverter = newXmlConverter(context);
         Document dom = xmlConverter.toDOMDocument(inputStream, null);
+        return loadRoutesDefinition(context, dom);
+    }
+
+    /**
+     * Marshal the xml to the model definition
+     *
+     * @param context the CamelContext, if <tt>null</tt> then {@link org.apache.camel.spi.ModelJAXBContextFactory} is not in use
+     * @param node the xml node
+     * @throws Exception is thrown if an error is encountered unmarshalling from xml to model
+     */
+    public static RoutesDefinition loadRoutesDefinition(CamelContext context, Node node) throws Exception {
+        JAXBContext jaxbContext = getJAXBContext(context);
 
         Map<String, String> namespaces = new LinkedHashMap<>();
-        extractNamespaces(dom, namespaces);
+        extractNamespaces(node.getOwnerDocument(), namespaces);
 
         Binder<Node> binder = jaxbContext.createBinder();
-        Object result = binder.unmarshal(dom);
+        Object result = binder.unmarshal(node);
 
         if (result == null) {
             throw new JAXBException("Cannot unmarshal to RoutesDefinition using JAXB");

http://git-wip-us.apache.org/repos/asf/camel/blob/fbbf8299/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
index e0957d7..7dbc14b 100644
--- a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
@@ -19,12 +19,16 @@ package org.apache.camel.support;
 import java.io.InputStream;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.model.ModelHelper;
 import org.apache.camel.model.RoutesDefinition;
 import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
 
 /**
  * Base class for implementing custom {@link ReloadStrategy} SPI plugins.
@@ -63,21 +67,37 @@ public abstract class ReloadStrategySupport extends ServiceSupport implements Re
 
     @Override
     public void onReloadRoutes(CamelContext camelContext, String name, InputStream resource) {
-
-        // load the stream in as DOM and find out if its <routes> <route> or <camelContext>
-        // and if its <blueprint> <beans> etc and then find inside the <camelContext> and grab what we support re-loading
-
         log.debug("Reloading CamelContext: {} routes from resource: {}", camelContext.getName(), name);
-        // assume the resource is XML routes
-        try {
-            RoutesDefinition routes = camelContext.loadRoutesDefinition(resource);
-            camelContext.addRouteDefinitions(routes.getRoutes());
-        } catch (Exception e) {
+
+        Document dom = camelContext.getTypeConverter().tryConvertTo(Document.class, resource);
+        if (dom == null) {
             failed++;
-            throw ObjectHelper.wrapRuntimeCamelException(e);
+            log.warn("Cannot load the resource " + name + " as XML");
+            return;
         }
-        log.info("Reloaded CamelContext: {} routes from resource: {}", camelContext.getName(), name);
 
+        // find the <routes> root
+        NodeList list = dom.getElementsByTagName("routes");
+        if (list == null || list.getLength() == 0) {
+            // fallback to <route>
+            list = dom.getElementsByTagName("route");
+        }
+
+        if (list != null && list.getLength() > 0) {
+            for (int i = 0; i < list.getLength(); i++) {
+                Node node = list.item(i);
+                try {
+                    RoutesDefinition routes = ModelHelper.loadRoutesDefinition(camelContext, node);
+                    camelContext.addRouteDefinitions(routes.getRoutes());
+                } catch (Exception e) {
+                    failed++;
+                    throw ObjectHelper.wrapRuntimeCamelException(e);
+                }
+
+            }
+        }
+
+        log.info("Reloaded CamelContext: {} routes from resource: {}", camelContext.getName(), name);
         succeeded++;
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/fbbf8299/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
----------------------------------------------------------------------
diff --git a/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml b/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
index ec81cc1..f6df299 100644
--- a/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
+++ b/examples/camel-example-reload/src/main/resources/META-INF/spring/camel-context.xml
@@ -23,13 +23,13 @@
        http://camel.apache.org/schema/spring http://camel.apache.org/schema/spring/camel-spring.xsd">
 
   <camelContext xmlns="http://camel.apache.org/schema/spring">
-    <route>
+    <route id="myRoute">
       <from uri="timer:foo"/>
       <!-- try to change me and save this file -->
       <transform>
         <constant>Hello World</constant>
       </transform>
-      <to uri="log:out"/>
+      <log message="You said: ${body}"/>
     </route>
   </camelContext>
 


[7/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: 76218fb1c4876dcfad484ea1dbf062799eeb768a
Parents: b868b5f
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 15:42:45 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../camel/impl/FileWatcherReloadStrategy.java   | 46 ++++++++------
 .../ManagedFileWatcherReloadStrategyTest.java   | 65 ++++++++++++++++++++
 2 files changed, 92 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/76218fb1/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
index e80f048..20573b7 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
@@ -27,13 +27,13 @@ import java.util.Locale;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
-import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
-
 import org.apache.camel.support.ReloadStrategySupport;
 import org.apache.camel.util.IOHelper;
 import org.apache.camel.util.ObjectHelper;
 
+import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
+import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
+
 /**
  * A file based {@link org.apache.camel.spi.ReloadStrategy} which watches a file folder
  * for modified files and reload on file changes.
@@ -45,8 +45,7 @@ import org.apache.camel.util.ObjectHelper;
  */
 public class FileWatcherReloadStrategy extends ReloadStrategySupport {
 
-    private Path folder;
-    private WatchService watcher;
+    private String folder;
     private ExecutorService executorService;
 
     public FileWatcherReloadStrategy() {
@@ -56,12 +55,9 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
         setFolder(directory);
     }
 
-    public Path getFolder() {
-        return folder;
-    }
 
     public void setFolder(String folder) {
-        this.folder = new File(folder).toPath();
+        this.folder = folder;
     }
 
     @Override
@@ -71,17 +67,21 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
             return;
         }
 
-        log.info("Starting ReloadStrategy to watch directory: {}", folder);
-        try {
-            this.watcher = folder.getFileSystem().newWatchService();
-            // we cannot support deleting files as we don't know which routes that would be
-            folder.register(watcher, ENTRY_CREATE, ENTRY_MODIFY);
-        } catch (IOException e) {
-            throw ObjectHelper.wrapRuntimeCamelException(e);
+        File dir = new File(folder);
+        if (dir.exists() && dir.isDirectory()) {
+            log.info("Starting ReloadStrategy to watch directory: {}", dir);
+            try {
+                Path path = dir.toPath();
+                WatchService watcher = path.getFileSystem().newWatchService();
+                // we cannot support deleting files as we don't know which routes that would be
+                path.register(watcher, ENTRY_CREATE, ENTRY_MODIFY);
+
+                executorService = getCamelContext().getExecutorServiceManager().newSingleThreadExecutor(this, "FileWatcherReloadStrategy");
+                executorService.submit(new WatchFileChangesTask(watcher, path));
+            } catch (IOException e) {
+                throw ObjectHelper.wrapRuntimeCamelException(e);
+            }
         }
-
-        executorService = getCamelContext().getExecutorServiceManager().newSingleThreadExecutor(this, "FileWatcherReloadStrategy");
-        executorService.submit(new WatchFileChangesTask());
     }
 
 
@@ -98,6 +98,14 @@ public class FileWatcherReloadStrategy extends ReloadStrategySupport {
      */
     protected class WatchFileChangesTask implements Runnable {
 
+        private final WatchService watcher;
+        private final Path folder;
+
+        public WatchFileChangesTask(WatchService watcher, Path folder) {
+            this.watcher = watcher;
+            this.folder = folder;
+        }
+
         public void run() {
             log.debug("ReloadStrategy is starting watching folder: {}", folder);
 

http://git-wip-us.apache.org/repos/asf/camel/blob/76218fb1/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java b/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
new file mode 100644
index 0000000..76e7aa5
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/management/ManagedFileWatcherReloadStrategyTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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.management;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.FileWatcherReloadStrategy;
+
+/**
+ * @version 
+ */
+public class ManagedFileWatcherReloadStrategyTest extends ManagementTestSupport {
+
+    public void testReloadStrategy() throws Exception {
+        // JMX tests dont work well on AIX CI servers (hangs them)
+        if (isPlatform("aix")) {
+            return;
+        }
+
+        MBeanServer mbeanServer = getMBeanServer();
+
+        ObjectName on = ObjectName.getInstance("org.apache.camel:context=camel-1,type=services,name=FileWatcherReloadStrategy");
+        assertTrue(mbeanServer.isRegistered(on));
+
+        String strategy = (String) mbeanServer.getAttribute(on, "Strategy");
+        assertEquals("FileWatcherReloadStrategy", strategy);
+
+        Integer reload = (Integer) mbeanServer.getAttribute(on, "ReloadCounter");
+        assertEquals(0, reload.intValue());
+
+        Integer failed = (Integer) mbeanServer.getAttribute(on, "FailedCounter");
+        assertEquals(0, failed.intValue());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // add reload strategy
+                context.setReloadStrategy(new FileWatcherReloadStrategy("target/dummy"));
+
+                from("direct:start")
+                    .to("mock:result");
+            }
+        };
+    }
+
+}


[5/8] camel git commit: CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.

Posted by da...@apache.org.
CAMEL-10599: Add ReloadStrategy to allow watching for file changes and reload routes on the fly.


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

Branch: refs/heads/master
Commit: d9fb32e96369d0d3c648fc7b243115af7ce38207
Parents: ec735be
Author: Claus Ibsen <da...@apache.org>
Authored: Mon Dec 19 13:45:13 2016 +0100
Committer: Claus Ibsen <da...@apache.org>
Committed: Mon Dec 19 16:34:46 2016 +0100

----------------------------------------------------------------------
 .../java/org/apache/camel/CamelContext.java     |  13 ++
 .../apache/camel/impl/DefaultCamelContext.java  | 127 +++++++++--------
 .../camel/impl/FileWatcherReloadStrategy.java   | 132 ++++++++++++++++++
 .../java/org/apache/camel/main/MainSupport.java |  10 ++
 .../org/apache/camel/spi/ReloadStrategy.java    |  49 +++++++
 .../camel/support/ReloadStrategySupport.java    |  70 ++++++++++
 .../impl/FileWatcherReloadStrategyTest.java     | 135 +++++++++++++++++++
 camel-core/src/test/resources/log4j2.properties |  39 +-----
 8 files changed, 485 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/CamelContext.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/CamelContext.java b/camel-core/src/main/java/org/apache/camel/CamelContext.java
index dab1471..c827748 100644
--- a/camel-core/src/main/java/org/apache/camel/CamelContext.java
+++ b/camel-core/src/main/java/org/apache/camel/CamelContext.java
@@ -63,6 +63,7 @@ import org.apache.camel.spi.NodeIdFactory;
 import org.apache.camel.spi.PackageScanClassResolver;
 import org.apache.camel.spi.ProcessorFactory;
 import org.apache.camel.spi.Registry;
+import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestRegistry;
 import org.apache.camel.spi.RoutePolicyFactory;
@@ -1843,4 +1844,16 @@ public interface CamelContext extends SuspendableService, RuntimeConfiguration {
      */
     void setModelJAXBContextFactory(ModelJAXBContextFactory modelJAXBContextFactory);
 
+    /**
+     * Returns the {@link ReloadStrategy} if in use.
+     *
+     * @return the strategy, or <tt>null</tt> if none has been configured.
+     */
+    ReloadStrategy getReloadStrategy();
+
+    /**
+     * Sets a custom {@link ReloadStrategy} to be used
+     */
+    void setReloadStrategy(ReloadStrategy reloadStrategy);
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
index 3a16975..a9b30a4 100644
--- a/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
+++ b/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
@@ -145,6 +145,7 @@ import org.apache.camel.spi.NodeIdFactory;
 import org.apache.camel.spi.PackageScanClassResolver;
 import org.apache.camel.spi.ProcessorFactory;
 import org.apache.camel.spi.Registry;
+import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.spi.RestConfiguration;
 import org.apache.camel.spi.RestRegistry;
 import org.apache.camel.spi.RouteContext;
@@ -283,6 +284,7 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
     private ModelJAXBContextFactory modelJAXBContextFactory;
     private List<TransformerDefinition> transformers = new ArrayList<TransformerDefinition>();
     private Map<TransformerKey, Transformer> transformerRegistry = new HashMap<TransformerKey, Transformer>();
+    private ReloadStrategy reloadStrategy;
 
     /**
      * Creates the {@link CamelContext} using {@link JndiRegistry} as registry,
@@ -3118,6 +3120,10 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
         addService(packageScanClassResolver, true, true);
         addService(restRegistry, true, true);
         addService(messageHistoryFactory, true, true);
+        if (reloadStrategy != null) {
+            log.info("Using ReloadStrategy: {}", reloadStrategy);
+            addService(reloadStrategy, true, true);
+        }
 
         if (runtimeEndpointRegistry != null) {
             if (runtimeEndpointRegistry instanceof EventNotifier) {
@@ -4247,64 +4253,14 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
         return value;
     }
 
-    protected Map<String, RouteService> getRouteServices() {
-        return routeServices;
-    }
-
-    protected ManagementStrategy createManagementStrategy() {
-        return new ManagementStrategyFactory().create(this, disableJMX || Boolean.getBoolean(JmxSystemPropertyKeys.DISABLED));
-    }
-
-    /**
-     * Reset context counter to a preset value. Mostly used for tests to ensure a predictable getName()
-     *
-     * @param value new value for the context counter
-     */
-    public static void setContextCounter(int value) {
-        DefaultCamelContextNameStrategy.setCounter(value);
-        DefaultManagementNameStrategy.setCounter(value);
-    }
-
-    private static UuidGenerator createDefaultUuidGenerator() {
-        if (System.getProperty("com.google.appengine.runtime.environment") != null) {
-            // either "Production" or "Development"
-            return new JavaUuidGenerator();
-        } else {
-            return new ActiveMQUuidGenerator();
-        }
-    }
-
-    protected ModelJAXBContextFactory createModelJAXBContextFactory() {
-        return new DefaultModelJAXBContextFactory();
-    }
-
     @Override
-    public String toString() {
-        return "CamelContext(" + getName() + ")";
+    public ReloadStrategy getReloadStrategy() {
+        return reloadStrategy;
     }
 
-    class MDCHelper implements AutoCloseable {
-        final Map<String, String> originalContextMap;
-
-        MDCHelper() {
-            if (isUseMDCLogging()) {
-                originalContextMap = MDC.getCopyOfContextMap();
-                MDC.put(MDC_CAMEL_CONTEXT_ID, getName());
-            } else {
-                originalContextMap = null;
-            }
-        }
-
-        @Override
-        public void close() {
-            if (isUseMDCLogging()) {
-                if (originalContextMap != null) {
-                    MDC.setContextMap(originalContextMap);
-                } else {
-                    MDC.clear();
-                }
-            }
-        }
+    @Override
+    public void setReloadStrategy(ReloadStrategy reloadStrategy) {
+        this.reloadStrategy = reloadStrategy;
     }
 
     @Override
@@ -4333,6 +4289,37 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
         return resolveTransformer(getTransformerKey(from, to));
     }
 
+    protected Map<String, RouteService> getRouteServices() {
+        return routeServices;
+    }
+
+    protected ManagementStrategy createManagementStrategy() {
+        return new ManagementStrategyFactory().create(this, disableJMX || Boolean.getBoolean(JmxSystemPropertyKeys.DISABLED));
+    }
+
+    /**
+     * Reset context counter to a preset value. Mostly used for tests to ensure a predictable getName()
+     *
+     * @param value new value for the context counter
+     */
+    public static void setContextCounter(int value) {
+        DefaultCamelContextNameStrategy.setCounter(value);
+        DefaultManagementNameStrategy.setCounter(value);
+    }
+
+    private static UuidGenerator createDefaultUuidGenerator() {
+        if (System.getProperty("com.google.appengine.runtime.environment") != null) {
+            // either "Production" or "Development"
+            return new JavaUuidGenerator();
+        } else {
+            return new ActiveMQUuidGenerator();
+        }
+    }
+
+    protected ModelJAXBContextFactory createModelJAXBContextFactory() {
+        return new DefaultModelJAXBContextFactory();
+    }
+
     protected Transformer resolveTransformer(TransformerKey key) {
         Transformer transformer = transformerRegistry.get(key);
         if (transformer != null) {
@@ -4362,4 +4349,34 @@ public class DefaultCamelContext extends ServiceSupport implements ModelCamelCon
     protected TransformerKey getTransformerKey(DataType from, DataType to) {
         return new TransformerKey(from, to);
     }
+
+    @Override
+    public String toString() {
+        return "CamelContext(" + getName() + ")";
+    }
+
+    class MDCHelper implements AutoCloseable {
+        final Map<String, String> originalContextMap;
+
+        MDCHelper() {
+            if (isUseMDCLogging()) {
+                originalContextMap = MDC.getCopyOfContextMap();
+                MDC.put(MDC_CAMEL_CONTEXT_ID, getName());
+            } else {
+                originalContextMap = null;
+            }
+        }
+
+        @Override
+        public void close() {
+            if (isUseMDCLogging()) {
+                if (originalContextMap != null) {
+                    MDC.setContextMap(originalContextMap);
+                } else {
+                    MDC.clear();
+                }
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
new file mode 100644
index 0000000..f7bb58a
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/impl/FileWatcherReloadStrategy.java
@@ -0,0 +1,132 @@
+/**
+ * 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.impl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.WatchEvent;
+import java.nio.file.WatchKey;
+import java.nio.file.WatchService;
+import java.util.Locale;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardWatchEventKinds.ENTRY_CREATE;
+import static java.nio.file.StandardWatchEventKinds.ENTRY_MODIFY;
+
+import org.apache.camel.support.ReloadStrategySupport;
+import org.apache.camel.util.IOHelper;
+import org.apache.camel.util.ObjectHelper;
+
+/**
+ * A file based {@link org.apache.camel.spi.ReloadStrategy} which watches a file folder
+ * for modified files and reload on file changes.
+ */
+public class FileWatcherReloadStrategy extends ReloadStrategySupport {
+
+    // TODO: support multiple folders
+
+    private Path folder;
+    private WatchService watcher;
+    private ExecutorService executorService;
+
+    public FileWatcherReloadStrategy() {
+    }
+
+    public Path getFolder() {
+        return folder;
+    }
+
+    public void setFolder(String folder) {
+        this.folder = new File(folder).toPath();
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        log.info("Starting ReloadStrategy to watch directory: {}", folder);
+        try {
+            this.watcher = folder.getFileSystem().newWatchService();
+            // we cannot support deleting files as we don't know which routes that would be
+            folder.register(watcher, ENTRY_CREATE, ENTRY_MODIFY);
+        } catch (IOException e) {
+            throw ObjectHelper.wrapRuntimeCamelException(e);
+        }
+
+        executorService = getCamelContext().getExecutorServiceManager().newSingleThreadExecutor(this, "FileWatcherReloadStrategy");
+        executorService.submit(new WatchFileChangesTask());
+    }
+
+
+    @Override
+    protected void doStop() throws Exception {
+        getCamelContext().getExecutorServiceManager().shutdownGraceful(executorService);
+    }
+
+    /**
+     * Background task which watches for file changes
+     */
+    protected class WatchFileChangesTask implements Runnable {
+
+        public void run() {
+            log.debug("ReloadStrategy is starting watching folder: {}", folder);
+
+            // allow to run while starting Camel
+            while (isStarting() || isRunAllowed()) {
+                WatchKey key;
+                try {
+                    log.trace("ReloadStrategy is polling for file changes in directory: {}", folder);
+                    // wait for a key to be available
+                    key = watcher.poll(2, TimeUnit.SECONDS);
+                } catch (InterruptedException ex) {
+                    break;
+                }
+
+                if (key != null) {
+                    for (WatchEvent<?> event : key.pollEvents()) {
+                        WatchEvent<Path> we = (WatchEvent<Path>) event;
+                        Path path = we.context();
+                        String name = folder.resolve(path).toAbsolutePath().toFile().getAbsolutePath();
+                        log.trace("Modified/Created file: {}", name);
+
+                        // must be an .xml file
+                        if (name.toLowerCase(Locale.US).endsWith(".xml")) {
+                            log.debug("Modified/Created XML file: {}", name);
+                            try {
+                                FileInputStream fis = new FileInputStream(name);
+                                onReloadRoutes(getCamelContext(), name, fis);
+                                IOHelper.close(fis);
+                            } catch (Exception e) {
+                                log.warn("Error reloading routes from file: " + name + " due " + e.getMessage() + ". This exception is ignored.", e);
+                            }
+                        }
+                    }
+
+                    // the key must be reset after processed
+                    boolean valid = key.reset();
+                    if (!valid) {
+                        break;
+                    }
+                }
+            }
+
+            log.info("ReloadStrategy is stopping watching folder: {}", folder);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
index 74a22d4..dc7bc97 100644
--- a/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
+++ b/camel-core/src/main/java/org/apache/camel/main/MainSupport.java
@@ -35,6 +35,7 @@ import org.apache.camel.impl.DefaultModelJAXBContextFactory;
 import org.apache.camel.model.ModelCamelContext;
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.spi.ModelJAXBContextFactory;
+import org.apache.camel.spi.ReloadStrategy;
 import org.apache.camel.support.ServiceSupport;
 import org.apache.camel.util.ServiceHelper;
 import org.slf4j.Logger;
@@ -63,6 +64,7 @@ public abstract class MainSupport extends ServiceSupport {
     protected ProducerTemplate camelTemplate;
     protected boolean hangupInterceptorEnabled = true;
     protected int durationHitExitCode = DEFAULT_EXIT_CODE;
+    protected ReloadStrategy reloadStrategy;
 
     /**
      * A class for intercepting the hang up signal and do a graceful shutdown of the Camel.
@@ -348,6 +350,14 @@ public abstract class MainSupport extends ServiceSupport {
         return routeBuilderClasses;
     }
 
+    public ReloadStrategy getReloadStrategy() {
+        return reloadStrategy;
+    }
+
+    public void setReloadStrategy(ReloadStrategy reloadStrategy) {
+        this.reloadStrategy = reloadStrategy;
+    }
+
     public boolean isTrace() {
         return trace;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
new file mode 100644
index 0000000..7f30552
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/spi/ReloadStrategy.java
@@ -0,0 +1,49 @@
+/**
+ * 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.spi;
+
+import java.io.InputStream;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Service;
+import org.apache.camel.StaticService;
+
+/**
+ * SPI strategy for reloading Camel routes in an existing running {@link org.apache.camel.CamelContext}
+ */
+public interface ReloadStrategy extends Service, StaticService, CamelContextAware {
+
+    // TODO: naming of this SPI?
+    // TODO: Add JMX mbean
+
+    /**
+     * A reload is triggered and the {@link CamelContext} is expected to do a re-start
+     *
+     * @param camelContext  the running CamelContext
+     */
+    void onReloadCamelContext(CamelContext camelContext);
+
+    /**
+     * A reload is triggered with a resource of some sort has been changed, such as a xml file with Camel routes.
+     *
+     * @param camelContext  the running CamelContext
+     * @param name          name of resource such as a file name (can be null)
+     * @param resource      the changed resource
+     */
+    void onReloadRoutes(CamelContext camelContext, String name, InputStream resource);
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.java
new file mode 100644
index 0000000..355f19d
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/support/ReloadStrategySupport.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.support;
+
+import java.io.InputStream;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.model.RoutesDefinition;
+import org.apache.camel.spi.ReloadStrategy;
+import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.ServiceHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for implementing custom {@link ReloadStrategy} SPI plugins.
+ */
+public abstract class ReloadStrategySupport extends ServiceSupport implements ReloadStrategy {
+    protected final Logger log = LoggerFactory.getLogger(getClass());
+    private CamelContext camelContext;
+
+    @Override
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    @Override
+    public void onReloadCamelContext(CamelContext camelContext) {
+        log.debug("Reloading CamelContext: {}", camelContext.getName());
+        try {
+            ServiceHelper.stopService(camelContext);
+            ServiceHelper.startService(camelContext);
+        } catch (Exception e) {
+            throw ObjectHelper.wrapRuntimeCamelException(e);
+        }
+        log.info("Reloaded CamelContext: {}", camelContext.getName());
+    }
+
+    @Override
+    public void onReloadRoutes(CamelContext camelContext, String name, InputStream resource) {
+        log.debug("Reloading CamelContext: {} routes from resource: {}", camelContext.getName(), name);
+        // assume the resource is XML routes
+        try {
+            RoutesDefinition routes = camelContext.loadRoutesDefinition(resource);
+            camelContext.addRouteDefinitions(routes.getRoutes());
+        } catch (Exception e) {
+            throw ObjectHelper.wrapRuntimeCamelException(e);
+        }
+        log.info("Reloaded CamelContext: {} routes from resource: {}", camelContext.getName(), name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/test/java/org/apache/camel/impl/FileWatcherReloadStrategyTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/impl/FileWatcherReloadStrategyTest.java b/camel-core/src/test/java/org/apache/camel/impl/FileWatcherReloadStrategyTest.java
new file mode 100644
index 0000000..6136712
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/impl/FileWatcherReloadStrategyTest.java
@@ -0,0 +1,135 @@
+/**
+ * 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.impl;
+
+import java.io.File;
+import java.util.EventObject;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.management.event.RouteAddedEvent;
+import org.apache.camel.support.EventNotifierSupport;
+import org.apache.camel.util.FileUtil;
+
+public class FileWatcherReloadStrategyTest extends ContextTestSupport {
+
+    private FileWatcherReloadStrategy reloadStrategy;
+
+    @Override
+    public boolean isUseRouteBuilder() {
+        return false;
+    }
+
+    @Override
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext context = super.createCamelContext();
+        reloadStrategy = new FileWatcherReloadStrategy();
+        reloadStrategy.setFolder("target/dummy");
+        context.setReloadStrategy(reloadStrategy);
+        return context;
+    }
+
+    public void testAddNewRoute() throws Exception {
+        deleteDirectory("target/dummy");
+        createDirectory("target/dummy");
+
+        context.start();
+
+        // there are 0 routes to begin with
+        assertEquals(0, context.getRoutes().size());
+
+        Thread.sleep(1000);
+        log.info("Copying file to target/dummy");
+
+        // create an xml file with some routes
+        FileUtil.copyFile(new File("src/test/resources/org/apache/camel/model/barRoute.xml"), new File("target/dummy/barRoute.xml"));
+
+        // wait for that file to be processed
+        // (is slow on osx, so wait up till 20 seconds)
+        for (int i = 0; i < 20; i++) {
+            if (context.getRoutes().size() > 0) {
+                break;
+            }
+            Thread.sleep(1000);
+        }
+
+        assertEquals(1, context.getRoutes().size());
+
+        // and the route should work
+        getMockEndpoint("mock:bar").expectedMessageCount(1);
+        template.sendBody("direct:bar", "Hello World");
+        assertMockEndpointsSatisfied();
+    }
+
+    public void testUpdateExistingRoute() throws Exception {
+        deleteDirectory("target/dummy");
+        createDirectory("target/dummy");
+
+        // the bar route is added two times, at first, and then when updated
+        final CountDownLatch latch = new CountDownLatch(2);
+        context.getManagementStrategy().addEventNotifier(new EventNotifierSupport() {
+            @Override
+            public void notify(EventObject event) throws Exception {
+                latch.countDown();
+            }
+
+            @Override
+            public boolean isEnabled(EventObject event) {
+                System.out.println(event);
+                return event instanceof RouteAddedEvent;
+            }
+        });
+
+        context.addRoutes(new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:bar").routeId("bar").to("mock:foo");
+            }
+        });
+
+        context.start();
+
+        assertEquals(1, context.getRoutes().size());
+
+        // and the route should work sending to mock:foo
+        getMockEndpoint("mock:bar").expectedMessageCount(0);
+        getMockEndpoint("mock:foo").expectedMessageCount(1);
+        template.sendBody("direct:bar", "Hello World");
+        assertMockEndpointsSatisfied();
+
+        Thread.sleep(1000);
+        log.info("Copying file to target/dummy");
+
+        // create an xml file with some routes
+        FileUtil.copyFile(new File("src/test/resources/org/apache/camel/model/barRoute.xml"), new File("target/dummy/barRoute.xml"));
+
+        // wait for that file to be processed and remove/add the route
+        // (is slow on osx, so wait up till 20 seconds)
+        latch.await(20, TimeUnit.SECONDS);
+
+        resetMocks();
+
+        // and the route should be changed to route to mock:bar instead of mock:foo
+        getMockEndpoint("mock:bar").expectedMessageCount(1);
+        getMockEndpoint("mock:foo").expectedMessageCount(0);
+        template.sendBody("direct:bar", "Bye World");
+        assertMockEndpointsSatisfied();
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/d9fb32e9/camel-core/src/test/resources/log4j2.properties
----------------------------------------------------------------------
diff --git a/camel-core/src/test/resources/log4j2.properties b/camel-core/src/test/resources/log4j2.properties
index 7418c25..f066246 100644
--- a/camel-core/src/test/resources/log4j2.properties
+++ b/camel-core/src/test/resources/log4j2.properties
@@ -39,40 +39,9 @@ logger.customlogger.level = TRACE
 logger.customlogger.appenderRef.file2.ref = file2
 
 rootLogger.level = INFO
-rootLogger.appenderRef.file.ref = file
-
 
-#log4j.logger.org.apache.camel.impl.converter=WARN
-#log4j.logger.org.apache.camel.management=DEBUG
-#log4j.logger.org.apache.camel.impl.DefaultPackageScanClassResolver=WARN
-#log4j.logger.org.apache.camel.impl.converter.DefaultTypeConverter=TRACE
-#log4j.logger.org.apache.camel.impl.converter=DEBUG
-#log4j.logger.org.apache.camel=DEBUG
-#log4j.logger.org.apache.camel.builder=TRACE
-#log4j.logger.org.apache.camel.language.simple=TRACE
-#log4j.logger.org.apache.camel.component=TRACE
-#log4j.logger.org.apache.camel.component.seda=TRACE
-#log4j.logger.org.apache.camel.component.file=TRACE
-#log4j.logger.org.apache.camel.impl.DefaultUnitOfWork=TRACE
-#log4j.logger.org.apache.camel.impl.DefaultExecutorServiceManager=TRACE
-#log4j.logger.org.apache.camel.impl.DefaultShutdownStrategy=DEBUG
-#log4j.logger.org.apache.camel.component.mock=DEBUG
-#log4j.logger.org.apache.camel.component.file=TRACE
-#log4j.logger.org.apache.camel.processor.DefaultErrorHandler=TRACE
-#log4j.logger.org.apache.camel.processor.DeadLetterChannel=TRACE
-#log4j.logger.org.apache.camel.processor.Pipeline=TRACE
-#log4j.logger.org.apache.camel.processor.MulticastProcessor=TRACE
-#log4j.logger.org.apache.camel.processor.RecipientList=TRACE
-#log4j.logger.org.apache.camel.processor.RecipientListProcessor=TRACE
-#log4j.logger.org.apache.camel.processor.RoutingSlip=TRACE
-#log4j.logger.org.apache.camel.processor.TryProcessor=TRACE
-#log4j.logger.org.apache.camel.processor.loadbalancer=TRACE
-#log4j.logger.org.apache.camel.processor.Delayer=TRACE
-#log4j.logger.org.apache.camel.processor.Throttler=TRACE
-#log4j.logger.org.apache.camel.processor.aggregate.AggregateProcessor=TRACE
-#log4j.logger.org.apache.camel.impl=TRACE
-#log4j.logger.org.apache.camel.util.FileUtil=TRACE
-#log4j.logger.org.apache.camel.util.AsyncProcessorHelper=TRACE
-#log4j.logger.org.apache.camel.util.ServiceHelper=TRACE
-#log4j.logger.org.apache.camel.util.jsse=TRACE
+rootLogger.appenderRef.file.ref = file
+# rootLogger.appenderRef.file.ref = console
 
+#logger.camel-core.name = org.apache.camel.impl
+#logger.camel-core.level = DEBUG