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 2010/07/08 09:43:36 UTC

svn commit: r961615 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/impl/ main/java/org/apache/camel/management/event/ main/java/org/apache/camel/processor/interceptor/ main/java/org/apache/camel/spi/ test/java/org/apache/camel/processor/i...

Author: davsclaus
Date: Thu Jul  8 07:43:35 2010
New Revision: 961615

URL: http://svn.apache.org/viewvc?rev=961615&view=rev
Log:
CAMEL-2919: Debugger API

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java   (with props)
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java   (contents, props changed)
      - copied, changed from r961578, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCompletedEvent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCreatedEvent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureEvent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureHandledEvent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeSentEvent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debug.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Breakpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Condition.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Debugger.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java Thu Jul  8 07:43:35 2010
@@ -16,13 +16,17 @@
  */
 package org.apache.camel.impl;
 
+import java.util.EventObject;
+
 import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.Breakpoint;
 
 /**
  * A support class for {@link Breakpoint} implementations to use as base class.
  * <p/>
- * Will be in active state and match any {@link Exchange}s.
+ * Will be in active state.
  *
  * @version $Revision$
  */
@@ -42,4 +46,15 @@ public abstract class BreakpointSupport 
         state = State.Active;
     }
 
+    public void beforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+        // noop
+    }
+
+    public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+        // noop
+    }
+
+    public void onEvent(Exchange exchange, EventObject event, ProcessorDefinition definition) {
+        // noop
+    }
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java?rev=961615&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java Thu Jul  8 07:43:35 2010
@@ -0,0 +1,40 @@
+/**
+ * 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.util.EventObject;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.spi.Condition;
+
+/**
+ * A support class for {@link org.apache.camel.spi.Condition} implementations to use as base class.
+ *
+ * @version $Revision$
+ */
+public class ConditionSupport implements Condition {
+
+    public boolean matchProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+        return false;
+    }
+
+    public boolean matchEvent(Exchange exchange, EventObject event) {
+        return false;
+    }
+}

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ConditionSupport.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java Thu Jul  8 07:43:35 2010
@@ -1023,8 +1023,12 @@ public class DefaultCamelContext extends
             }
         }
         
+        // register debugger
         if (getDebugger() != null) {
             LOG.info("Debugger: " + getDebugger() + " is enabled on CamelContext: " + getName());
+            // register this camel context on the debugger
+            getDebugger().setCamelContext(this);
+            startServices(getDebugger());
             addInterceptStrategy(new Debug(getDebugger()));
         }
 
@@ -1073,6 +1077,7 @@ public class DefaultCamelContext extends
             routeDefinitionInitiated = true;
         }
 
+
         // starting will continue in the start method
     }
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.java Thu Jul  8 07:43:35 2010
@@ -19,14 +19,23 @@ package org.apache.camel.impl;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.EventObject;
 import java.util.List;
 
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
+import org.apache.camel.LoggingLevel;
 import org.apache.camel.Processor;
+import org.apache.camel.RouteNode;
+import org.apache.camel.management.EventNotifierSupport;
+import org.apache.camel.management.event.AbstractExchangeEvent;
 import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.spi.Breakpoint;
 import org.apache.camel.spi.Condition;
 import org.apache.camel.spi.Debugger;
+import org.apache.camel.util.ObjectHelper;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -35,10 +44,11 @@ import org.apache.commons.logging.LogFac
  *
  * @version $Revision$
  */
-public class DefaultDebugger implements Debugger {
+public class DefaultDebugger implements Debugger, CamelContextAware {
 
     private static final Log LOG = LogFactory.getLog(DefaultDebugger.class);
     private final List<BreakpointConditions> breakpoints = new ArrayList<BreakpointConditions>();
+    private CamelContext camelContext;
 
     /**
      * Holder class for breakpoint and the associated conditions
@@ -65,6 +75,21 @@ public class DefaultDebugger implements 
         }
     }
 
+    public DefaultDebugger() {
+    }
+
+    public DefaultDebugger(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
     public void addBreakpoint(Breakpoint breakpoint) {
         breakpoints.add(new BreakpointConditions(breakpoint));
     }
@@ -97,7 +122,24 @@ public class DefaultDebugger implements 
         return Collections.unmodifiableList(answer);
     }
 
-    public boolean onExchange(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+    public boolean beforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+        boolean match = false;
+
+        // does any of the breakpoints apply?
+        for (BreakpointConditions breakpoint : breakpoints) {
+            // breakpoint must be active
+            if (Breakpoint.State.Active.equals(breakpoint.getBreakpoint().getState())) {
+                if (matchConditions(exchange, processor, definition, breakpoint)) {
+                    match = true;
+                    onBeforeProcess(exchange, processor, definition, breakpoint.getBreakpoint());
+                }
+            }
+        }
+
+        return match;
+    }
+
+    public boolean afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
         boolean match = false;
 
         // does any of the breakpoints apply?
@@ -106,7 +148,7 @@ public class DefaultDebugger implements 
             if (Breakpoint.State.Active.equals(breakpoint.getBreakpoint().getState())) {
                 if (matchConditions(exchange, processor, definition, breakpoint)) {
                     match = true;
-                    onBreakpoint(exchange, processor, definition, breakpoint.getBreakpoint());
+                    onAfterProcess(exchange, processor, definition, breakpoint.getBreakpoint());
                 }
             }
         }
@@ -114,10 +156,61 @@ public class DefaultDebugger implements 
         return match;
     }
 
-    private boolean matchConditions(Exchange exchange,  Processor processor, ProcessorDefinition definition, BreakpointConditions breakpoint) {
+    public boolean onEvent(Exchange exchange, EventObject event) {
+        boolean match = false;
+
+        // does any of the breakpoints apply?
+        for (BreakpointConditions breakpoint : breakpoints) {
+            // breakpoint must be active
+            if (Breakpoint.State.Active.equals(breakpoint.getBreakpoint().getState())) {
+                if (matchConditions(exchange, event, breakpoint)) {
+                    match = true;
+                    onEvent(exchange, event, breakpoint.getBreakpoint());
+                }
+            }
+        }
+
+        return match;
+    }
+
+    protected void onBeforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, Breakpoint breakpoint) {
+        try {
+            breakpoint.beforeProcess(exchange, processor, definition);
+        } catch (Throwable e) {
+            LOG.warn("Exception occurred in breakpoint: " + breakpoint + ". This exception will be ignored.", e);
+        }
+    }
+
+    protected void onAfterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, Breakpoint breakpoint) {
+        try {
+            breakpoint.afterProcess(exchange, processor, definition);
+        } catch (Throwable e) {
+            LOG.warn("Exception occurred in breakpoint: " + breakpoint + ". This exception will be ignored.", e);
+        }
+    }
+
+    protected void onEvent(Exchange exchange, EventObject event, Breakpoint breakpoint) {
+        ProcessorDefinition definition = null;
+
+        // try to get the last known definition
+        if (exchange.getUnitOfWork() != null && exchange.getUnitOfWork().getTracedRouteNodes() != null) {
+            RouteNode node = exchange.getUnitOfWork().getTracedRouteNodes().getLastNode();
+            if (node != null) {
+                definition = node.getProcessorDefinition();
+            }
+        }
+
+        try {
+            breakpoint.onEvent(exchange, event, definition);
+        } catch (Throwable e) {
+            LOG.warn("Exception occurred in breakpoint: " + breakpoint + ". This exception will be ignored.", e);
+        }
+    }
+
+    private boolean matchConditions(Exchange exchange, Processor processor, ProcessorDefinition definition, BreakpointConditions breakpoint) {
         if (breakpoint.getConditions() != null && !breakpoint.getConditions().isEmpty()) {
             for (Condition condition : breakpoint.getConditions()) {
-                if (!condition.match(exchange, definition)) {
+                if (!condition.matchProcess(exchange, processor, definition)) {
                     return false;
                 }
             }
@@ -126,21 +219,66 @@ public class DefaultDebugger implements 
         return true;
     }
 
-    protected void onBreakpoint(Exchange exchange, Processor processor, ProcessorDefinition definition, Breakpoint breakpoint) {
-        breakpoint.onExchange(exchange, processor, definition);
+    private boolean matchConditions(Exchange exchange, EventObject event, BreakpointConditions breakpoint) {
+        if (breakpoint.getConditions() != null && !breakpoint.getConditions().isEmpty()) {
+            for (Condition condition : breakpoint.getConditions()) {
+                if (!condition.matchEvent(exchange, event)) {
+                    return false;
+                }
+            }
+        }
+
+        return true;
     }
 
     public void start() throws Exception {
-        // noop
+        ObjectHelper.notNull(camelContext, "CamelContext", this);
+        // register our event notifier
+        camelContext.getManagementStrategy().addEventNotifier(new DebugEventNotifier());
+        Tracer tracer = Tracer.getTracer(camelContext);
+        if (tracer == null) {
+            // tracer is disabled so enable it silently so we can leverage it to trace the Exchanges for us
+            tracer = Tracer.createTracer(camelContext);
+            tracer.setLogLevel(LoggingLevel.OFF);
+            camelContext.addService(tracer);
+            camelContext.addInterceptStrategy(tracer);
+        }
     }
 
     public void stop() throws Exception {
         breakpoints.clear();
-        // noop
     }
 
     @Override
     public String toString() {
         return "DefaultDebugger";
     }
+
+    private final class DebugEventNotifier extends EventNotifierSupport {
+
+        private DebugEventNotifier() {
+            setIgnoreCamelContextEvents(true);
+            setIgnoreServiceEvents(true);
+        }
+
+        public void notify(EventObject event) throws Exception {
+            AbstractExchangeEvent aee = (AbstractExchangeEvent) event;
+            Exchange exchange = aee.getExchange();
+            onEvent(exchange, event);
+        }
+
+        public boolean isEnabled(EventObject event) {
+            return event instanceof AbstractExchangeEvent;
+        }
+
+        @Override
+        protected void doStart() throws Exception {
+            // noop
+        }
+
+        @Override
+        protected void doStop() throws Exception {
+            // noop
+        }
+    }
 }

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java?rev=961615&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java Thu Jul  8 07:43:35 2010
@@ -0,0 +1,40 @@
+/**
+ * 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.event;
+
+import java.util.EventObject;
+
+import org.apache.camel.Exchange;
+
+/**
+ * Base class for {@link Exchange} events.
+ *
+ * @version $Revision$
+ */
+public abstract class AbstractExchangeEvent extends EventObject {
+
+    private final Exchange exchange;
+
+    public AbstractExchangeEvent(Exchange source) {
+        super(source);
+        this.exchange = source;
+    }
+
+    public Exchange getExchange() {
+        return exchange;
+    }
+}

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/AbstractExchangeEvent.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCompletedEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCompletedEvent.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCompletedEvent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCompletedEvent.java Thu Jul  8 07:43:35 2010
@@ -16,29 +16,20 @@
  */
 package org.apache.camel.management.event;
 
-import java.util.EventObject;
-
 import org.apache.camel.Exchange;
 
 /**
  * @version $Revision$
  */
-public class ExchangeCompletedEvent extends EventObject {
+public class ExchangeCompletedEvent extends AbstractExchangeEvent {
     private static final long serialVersionUID = -3231801412021356098L;
 
-    private final Exchange exchange;
-
     public ExchangeCompletedEvent(Exchange source) {
         super(source);
-        this.exchange = source;
-    }
-
-    public Exchange getExchange() {
-        return exchange;
     }
 
     @Override
     public String toString() {
-        return exchange.getExchangeId() + " exchange completed: " + exchange;
+        return getExchange().getExchangeId() + " exchange completed: " + getExchange();
     }
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCreatedEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCreatedEvent.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCreatedEvent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeCreatedEvent.java Thu Jul  8 07:43:35 2010
@@ -16,29 +16,20 @@
  */
 package org.apache.camel.management.event;
 
-import java.util.EventObject;
-
 import org.apache.camel.Exchange;
 
 /**
  * @version $Revision$
  */
-public class ExchangeCreatedEvent extends EventObject {
+public class ExchangeCreatedEvent extends AbstractExchangeEvent {
     private static final long serialVersionUID = -19248832613958243L;
 
-    private final Exchange exchange;
-
     public ExchangeCreatedEvent(Exchange source) {
         super(source);
-        this.exchange = source;
-    }
-
-    public Exchange getExchange() {
-        return exchange;
     }
 
     @Override
     public String toString() {
-        return exchange.getExchangeId() + " exchange created: " + exchange;
+        return getExchange().getExchangeId() + " exchange created: " + getExchange();
     }
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureEvent.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureEvent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureEvent.java Thu Jul  8 07:43:35 2010
@@ -16,34 +16,25 @@
  */
 package org.apache.camel.management.event;
 
-import java.util.EventObject;
-
 import org.apache.camel.Exchange;
 
 /**
  * @version $Revision$
  */
-public class ExchangeFailureEvent extends EventObject {
+public class ExchangeFailureEvent extends AbstractExchangeEvent {
     private static final long serialVersionUID = -8484326904627268101L;
 
-    private final Exchange exchange;
-
     public ExchangeFailureEvent(Exchange source) {
         super(source);
-        this.exchange = source;
-    }
-
-    public Exchange getExchange() {
-        return exchange;
     }
 
     @Override
     public String toString() {
-        Exception cause = exchange.getException();
+        Exception cause = getExchange().getException();
         if (cause != null) {
-            return exchange.getExchangeId() + " exchange failure: " + exchange + " cause " + cause;
+            return getExchange().getExchangeId() + " exchange failure: " + getExchange() + " cause " + cause;
         } else {
-            return exchange.getExchangeId() + " exchange failure: " + exchange;
+            return getExchange().getExchangeId() + " exchange failure: " + getExchange();
         }
     }
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureHandledEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureHandledEvent.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureHandledEvent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeFailureHandledEvent.java Thu Jul  8 07:43:35 2010
@@ -16,32 +16,24 @@
  */
 package org.apache.camel.management.event;
 
-import java.util.EventObject;
-
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 
 /**
  * @version $Revision$
  */
-public class ExchangeFailureHandledEvent extends EventObject {
+public class ExchangeFailureHandledEvent extends AbstractExchangeEvent {
     private static final long serialVersionUID = -7554809462006009547L;
 
-    private final Exchange exchange;
     private final Processor failureHandler;
     private final boolean deadLetterChannel;
     private final boolean handled;
 
     public ExchangeFailureHandledEvent(Exchange source, Processor failureHandler, boolean deadLetterChannel) {
         super(source);
-        this.exchange = source;
         this.failureHandler = failureHandler;
         this.deadLetterChannel = deadLetterChannel;
-        this.handled = exchange.getProperty(Exchange.ERRORHANDLER_HANDLED, false, Boolean.class);
-    }
-
-    public Exchange getExchange() {
-        return exchange;
+        this.handled = source.getProperty(Exchange.ERRORHANDLER_HANDLED, false, Boolean.class);
     }
 
     public Processor getFailureHandler() {
@@ -59,9 +51,9 @@ public class ExchangeFailureHandledEvent
     @Override
     public String toString() {
         if (isDeadLetterChannel()) {
-            return exchange.getExchangeId() + " exchange failed: " + exchange + " but was handled by dead letter channel: " + failureHandler;
+            return getExchange().getExchangeId() + " exchange failed: " + getExchange() + " but was handled by dead letter channel: " + failureHandler;
         } else {
-            return exchange.getExchangeId() + " exchange failed: " + exchange + " but was processed by: " + failureHandler;
+            return getExchange().getExchangeId() + " exchange failed: " + getExchange() + " but was processed by: " + failureHandler;
         }
     }
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeSentEvent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeSentEvent.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeSentEvent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/event/ExchangeSentEvent.java Thu Jul  8 07:43:35 2010
@@ -16,32 +16,24 @@
  */
 package org.apache.camel.management.event;
 
-import java.util.EventObject;
-
 import org.apache.camel.Endpoint;
 import org.apache.camel.Exchange;
 
 /**
  * @version $Revision$
  */
-public class ExchangeSentEvent extends EventObject {
+public class ExchangeSentEvent extends AbstractExchangeEvent {
     private static final long serialVersionUID = -19248832613958123L;
 
-    private final Exchange exchange;
     private final Endpoint endpoint;
     private final long timeTaken;
 
     public ExchangeSentEvent(Exchange source, Endpoint endpoint, long timeTaken) {
         super(source);
-        this.exchange = source;
         this.endpoint = endpoint;
         this.timeTaken = timeTaken;
     }
 
-    public Exchange getExchange() {
-        return exchange;
-    }
-
     public Endpoint getEndpoint() {
         return endpoint;
     }
@@ -52,7 +44,7 @@ public class ExchangeSentEvent extends E
 
     @Override
     public String toString() {
-        return exchange.getExchangeId() + " exchange " + exchange + " sent to: " + endpoint.getEndpointUri() + " took: " + timeTaken + " ms.";
+        return getExchange().getExchangeId() + " exchange " + getExchange() + " sent to: " + endpoint.getEndpointUri() + " took: " + timeTaken + " ms.";
     }
 
 }
\ No newline at end of file

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debug.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debug.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debug.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/interceptor/Debug.java Thu Jul  8 07:43:35 2010
@@ -26,6 +26,8 @@ import org.apache.camel.spi.Debugger;
 import org.apache.camel.spi.InterceptStrategy;
 
 /**
+ * A debug interceptor to notify {@link Debugger} with {@link Exchange}s being processed.
+ *
  * @version $Revision$
  */
 public class Debug implements InterceptStrategy {
@@ -40,9 +42,16 @@ public class Debug implements InterceptS
                                                  final Processor target, final Processor nextTarget) throws Exception {
         return new DelegateAsyncProcessor(target) {
             @Override
-            public boolean process(Exchange exchange, AsyncCallback callback) {
-                debugger.onExchange(exchange, target, definition);
-                return super.process(exchange, callback);
+            public boolean process(final Exchange exchange, final AsyncCallback callback) {
+                debugger.beforeProcess(exchange, target, definition);
+
+                return super.process(exchange, new AsyncCallback() {
+                    public void done(boolean doneSync) {
+                        debugger.afterProcess(exchange, processor, definition);
+                        // must notify original callback
+                        callback.done(doneSync);
+                    }
+                });
             }
 
             @Override

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Breakpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Breakpoint.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Breakpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Breakpoint.java Thu Jul  8 07:43:35 2010
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.spi;
 
+import java.util.EventObject;
+
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.model.ProcessorDefinition;
@@ -26,20 +28,19 @@ import org.apache.camel.model.ProcessorD
  * This allows you to register {@link org.apache.camel.spi.Breakpoint}s to the {@link org.apache.camel.spi.Debugger}
  * and have those breakpoints activated when their {@link org.apache.camel.spi.Condition}s match.
  * <p/>
- * If any exceptions is thrown from the {@link #onExchange(org.apache.camel.Exchange, org.apache.camel.Processor, org.apache.camel.model.ProcessorDefinition)}
- * method then the {@link org.apache.camel.spi.Debugger} will catch and log those at <tt>WARN</tt> level and continue.
+ * If any exceptions is thrown from the callback methods then the {@link org.apache.camel.spi.Debugger}
+ * will catch and log those at <tt>WARN</tt> level and continue. This ensures Camel can continue to route
+ * the message without having breakpoints causing issues.
  *
+ * @version $Revision$
  * @see org.apache.camel.spi.Debugger
  * @see org.apache.camel.spi.Condition
- * @version $Revision$
  */
 public interface Breakpoint {
 
-    // TODO: Hook into the EventNotifier so we can have breakpoints trigger on those conditions as well
-    // exceptions, create, done, etc. and a FollowMe condition to follow a single exchange
-    // while others are being routed so you can follow one only, eg need an API on Debugger for that
-
-    enum State { Active, Suspended }
+    enum State {
+        Active, Suspended
+    }
 
     /**
      * Gets the state of this break
@@ -59,12 +60,32 @@ public interface Breakpoint {
     void activate();
 
     /**
-     * Callback invoked when the breakpoint was hit.
+     * Callback invoked when the breakpoint was hit and the {@link Exchange} is about to be processed (before).
+     *
+     * @param exchange   the {@link Exchange}
+     * @param processor  the {@link Processor} about to be processed
+     * @param definition the {@link org.apache.camel.model.ProcessorDefinition} definition of the processor
+     */
+    void beforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+
+    /**
+     * Callback invoked when the breakpoint was hit and the {@link Exchange} has been processed (after).
+     *
+     * @param exchange   the {@link Exchange}
+     * @param processor  the {@link Processor} which was processed
+     * @param definition the {@link org.apache.camel.model.ProcessorDefinition} definition of the processor
+     */
+    void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+
+    /**
+     * Callback invoked when the breakpoint was hit and any of the {@link Exchange} {@link EventObject event}s occurred.
      *
-     * @param exchange    the {@link Exchange}
-     * @param processor   the {@link Processor} which is the next target
-     * @param definition  the {@link org.apache.camel.model.ProcessorDefinition} definition of the processor
+     * @param exchange   the {@link Exchange}
+     * @param event      the event (instance of {@link org.apache.camel.management.event.AbstractExchangeEvent}
+     * @param definition the {@link org.apache.camel.model.ProcessorDefinition} definition of the last processor executed,
+     *                   may be <tt>null</tt> if not possible to resolve from tracing
+     * @see org.apache.camel.management.event.AbstractExchangeEvent
      */
-    void onExchange(Exchange exchange, Processor processor, ProcessorDefinition definition);
+    void onEvent(Exchange exchange, EventObject event, ProcessorDefinition definition);
 
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Condition.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Condition.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Condition.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Condition.java Thu Jul  8 07:43:35 2010
@@ -16,7 +16,10 @@
  */
 package org.apache.camel.spi;
 
+import java.util.EventObject;
+
 import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
 import org.apache.camel.model.ProcessorDefinition;
 
 /**
@@ -33,9 +36,20 @@ public interface Condition {
      * Does the condition match
      *
      * @param exchange the exchange
-     * @param definition the current node in the route where the Exchange is at
+     * @param processor  the {@link Processor}
+     * @param definition the present location in the route where the {@link Exchange} is located at
+     * @return <tt>true</tt> to match, <tt>false</tt> otherwise
+     */
+    boolean matchProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+
+    /**
+     * Does the condition match
+     *
+     * @param exchange the exchange
+     * @param event    the event (instance of {@link org.apache.camel.management.event.AbstractExchangeEvent}
      * @return <tt>true</tt> to match, <tt>false</tt> otherwise
+     * @see org.apache.camel.management.event.AbstractExchangeEvent
      */
-    boolean match(Exchange exchange, ProcessorDefinition definition);
+    boolean matchEvent(Exchange exchange, EventObject event);
 
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Debugger.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Debugger.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Debugger.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/Debugger.java Thu Jul  8 07:43:35 2010
@@ -16,8 +16,10 @@
  */
 package org.apache.camel.spi;
 
+import java.util.EventObject;
 import java.util.List;
 
+import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.Service;
@@ -29,7 +31,7 @@ import org.apache.camel.model.ProcessorD
  *
  * @version $Revision$
  */
-public interface Debugger extends Service {
+public interface Debugger extends Service, CamelContextAware {
 
     /**
      * Add the given breakpoint
@@ -71,14 +73,35 @@ public interface Debugger extends Servic
     List<Breakpoint> getBreakpoints();
 
     /**
+     * Callback invoked when an {@link Exchange} is about to be processed which allows implementators
+     * to notify breakpoints.
+     *
+     * @param exchange   the exchange
+     * @param processor  the {@link Processor} about to be processed
+     * @param definition the definition of the processor
+     * @return <tt>true</tt> if any breakpoint was hit, <tt>false</tt> if not breakpoint was hit
+     */
+    boolean beforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+
+    /**
+     * Callback invoked when an {@link Exchange} has been processed which allows implementators
+     * to notify breakpoints.
+     *
+     * @param exchange   the exchange
+     * @param processor  the {@link Processor} which was processed
+     * @param definition the definition of the processor
+     * @return <tt>true</tt> if any breakpoint was hit, <tt>false</tt> if not breakpoint was hit
+     */
+    boolean afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+
+    /**
      * Callback invoked when an {@link Exchange} is being processed which allows implementators
      * to notify breakpoints.
      *
-     * @param exchange     the exchange
-     * @param processor    the target processor (to be processed next)
-     * @param definition   the definition of the processor
+     * @param exchange the exchange
+     * @param event    the event (instance of {@link org.apache.camel.management.event.AbstractExchangeEvent}
      * @return <tt>true</tt> if any breakpoint was hit, <tt>false</tt> if not breakpoint was hit
      */
-    boolean onExchange(Exchange exchange, Processor processor, ProcessorDefinition definition);
+    boolean onEvent(Exchange exchange, EventObject event);
 
 }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java?rev=961615&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java Thu Jul  8 07:43:35 2010
@@ -0,0 +1,100 @@
+/**
+ * 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.processor.interceptor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.BreakpointSupport;
+import org.apache.camel.impl.ConditionSupport;
+import org.apache.camel.impl.DefaultDebugger;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.spi.Breakpoint;
+import org.apache.camel.spi.Condition;
+
+/**
+ * @version $Revision$
+ */
+public class DebugExceptionBreakpointTest extends ContextTestSupport {
+
+    private List<String> logs = new ArrayList<String>();
+    private Condition exceptionCondition;
+    private Breakpoint breakpoint;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+
+        breakpoint = new BreakpointSupport() {
+            @Override
+            public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+                Exception e = exchange.getException();
+                logs.add("Breakpoint at " + definition.getShortName() + " caused by: " + e.getClass().getSimpleName() + "[" + e.getMessage() + "]");
+            }
+        };
+
+        exceptionCondition = new ConditionSupport() {
+            @Override
+            public boolean matchProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+                return exchange.getException() != null;
+            }
+        };
+    }
+
+    public void testDebug() throws Exception {
+        context.getDebugger().addBreakpoint(breakpoint, exceptionCondition);
+
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World");
+
+        template.sendBody("direct:start", "Hello World");
+        try {
+            template.sendBody("direct:start", "Hello Camel");
+            fail("Should have thrown exception");
+        } catch (Exception e) {
+            // ignore
+        }
+
+        assertMockEndpointsSatisfied();
+
+        assertEquals(2, logs.size());
+        assertEquals("Breakpoint at when caused by: IllegalArgumentException[Damn]", logs.get(0));
+        assertEquals("Breakpoint at choice caused by: IllegalArgumentException[Damn]", logs.get(1));
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // use debugger
+                context.setDebugger(new DefaultDebugger());
+
+                from("direct:start")
+                    .to("log:foo")
+                    .choice()
+                        .when(body().contains("Camel")).throwException(new IllegalArgumentException("Damn"))
+                    .end()
+                    .to("mock:result");
+            }
+        };
+    }
+
+}
\ No newline at end of file

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java (from r961578, camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java&r1=961578&r2=961615&rev=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java Thu Jul  8 07:43:35 2010
@@ -17,27 +17,28 @@
 package org.apache.camel.processor.interceptor;
 
 import java.util.ArrayList;
+import java.util.EventObject;
 import java.util.List;
 
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.BreakpointSupport;
+import org.apache.camel.impl.ConditionSupport;
 import org.apache.camel.impl.DefaultDebugger;
+import org.apache.camel.management.event.AbstractExchangeEvent;
+import org.apache.camel.management.event.ExchangeFailureEvent;
 import org.apache.camel.model.ProcessorDefinition;
-import org.apache.camel.model.ToDefinition;
 import org.apache.camel.spi.Breakpoint;
 import org.apache.camel.spi.Condition;
 
 /**
  * @version $Revision$
  */
-public class DebugTest extends ContextTestSupport {
+public class DebugExceptionEventBreakpointTest extends ContextTestSupport {
 
     private List<String> logs = new ArrayList<String>();
-    private Condition camelCondition;
-    private Condition mockCondition;
+    private Condition exceptionCondition;
     private Breakpoint breakpoint;
 
     @Override
@@ -45,75 +46,37 @@ public class DebugTest extends ContextTe
         super.setUp();
 
         breakpoint = new BreakpointSupport() {
-            public void onExchange(Exchange exchange, Processor processor, ProcessorDefinition definition) {
-                String body = exchange.getIn().getBody(String.class);
-                logs.add("Breakpoint at " + definition + " with body: " + body);
+            public void onEvent(Exchange exchange, EventObject event, ProcessorDefinition definition) {
+                AbstractExchangeEvent aee = (AbstractExchangeEvent) event;
+                Exception e = aee.getExchange().getException();
+                logs.add("Breakpoint at " + definition + " caused by: " + e.getClass().getSimpleName() + "[" + e.getMessage() + "]");
             }
         };
 
-        camelCondition = new Condition() {
-            public boolean match(Exchange exchange, ProcessorDefinition definition) {
-                return body().contains("Camel").matches(exchange);
-            }
-        };
-
-        mockCondition = new Condition() {
-            public boolean match(Exchange exchange, ProcessorDefinition definition) {
-                // match when sending to mocks
-                if (definition instanceof ToDefinition) {
-                    ToDefinition to = (ToDefinition) definition;
-                    return to.getUriOrRef().startsWith("mock");
-                }
-                return false;
+        exceptionCondition = new ConditionSupport() {
+            public boolean matchEvent(Exchange exchange, EventObject event) {
+                return event instanceof ExchangeFailureEvent;
             }
         };
     }
 
     public void testDebug() throws Exception {
-        context.getDebugger().addBreakpoint(breakpoint, camelCondition);
-
-        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World", "Hello Camel");
-
-        template.sendBody("direct:start", "Hello World");
-        template.sendBody("direct:start", "Hello Camel");
-
-        assertMockEndpointsSatisfied();
-
-        assertEquals(2, logs.size());
-        assertEquals("Breakpoint at To[log:foo] with body: Hello Camel", logs.get(0));
-        assertEquals("Breakpoint at To[mock:result] with body: Hello Camel", logs.get(1));
-    }
-
-    public void testDebugSuspended() throws Exception {
-        context.getDebugger().addBreakpoint(breakpoint, mockCondition, camelCondition);
-
-        // suspend the breakpoint
-        context.getDebugger().suspendAllBreakpoints();
-
-        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World", "Hello Camel");
-
-        template.sendBody("direct:start", "Hello World");
-        template.sendBody("direct:start", "Hello Camel");
-
-        assertMockEndpointsSatisfied();
-
-        assertEquals(0, logs.size());
-
-        // resume the breakpoint
-        context.getDebugger().activateAllBreakpoints();
-
-        // reset and test again now the breakpoint is active
-        resetMocks();
+        context.getDebugger().addBreakpoint(breakpoint, exceptionCondition);
 
-        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World", "Hello Camel");
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World");
 
         template.sendBody("direct:start", "Hello World");
-        template.sendBody("direct:start", "Hello Camel");
+        try {
+            template.sendBody("direct:start", "Hello Camel");
+            fail("Should have thrown exception");
+        } catch (Exception e) {
+            // ignore
+        }
 
         assertMockEndpointsSatisfied();
 
         assertEquals(1, logs.size());
-        assertEquals("Breakpoint at To[mock:result] with body: Hello Camel", logs.get(0));
+        assertEquals("Breakpoint at ThrowException[java.lang.IllegalArgumentException] caused by: IllegalArgumentException[Damn]", logs.get(0));
     }
 
     @Override
@@ -124,9 +87,14 @@ public class DebugTest extends ContextTe
                 // use debugger
                 context.setDebugger(new DefaultDebugger());
 
-                from("direct:start").to("log:foo").to("mock:result");
+                from("direct:start")
+                    .to("log:foo")
+                    .choice()
+                        .when(body().contains("Camel")).throwException(new IllegalArgumentException("Damn"))
+                    .end()
+                    .to("mock:result");
             }
         };
     }
 
-}
+}
\ No newline at end of file

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionEventBreakpointTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java?rev=961615&r1=961614&r2=961615&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugTest.java Thu Jul  8 07:43:35 2010
@@ -17,6 +17,7 @@
 package org.apache.camel.processor.interceptor;
 
 import java.util.ArrayList;
+import java.util.EventObject;
 import java.util.List;
 
 import org.apache.camel.ContextTestSupport;
@@ -24,7 +25,9 @@ import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.BreakpointSupport;
+import org.apache.camel.impl.ConditionSupport;
 import org.apache.camel.impl.DefaultDebugger;
+import org.apache.camel.management.event.ExchangeCompletedEvent;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.model.ToDefinition;
 import org.apache.camel.spi.Breakpoint;
@@ -38,6 +41,7 @@ public class DebugTest extends ContextTe
     private List<String> logs = new ArrayList<String>();
     private Condition camelCondition;
     private Condition mockCondition;
+    private Condition doneCondition;
     private Breakpoint breakpoint;
 
     @Override
@@ -45,20 +49,25 @@ public class DebugTest extends ContextTe
         super.setUp();
 
         breakpoint = new BreakpointSupport() {
-            public void onExchange(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+            public void beforeProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
                 String body = exchange.getIn().getBody(String.class);
                 logs.add("Breakpoint at " + definition + " with body: " + body);
             }
+
+            public void onEvent(Exchange exchange, EventObject event, ProcessorDefinition definition) {
+                String body = exchange.getIn().getBody(String.class);
+                logs.add("Breakpoint event " + event.getClass().getSimpleName() + " with body: " + body);
+            }
         };
 
-        camelCondition = new Condition() {
-            public boolean match(Exchange exchange, ProcessorDefinition definition) {
+        camelCondition = new ConditionSupport() {
+            public boolean matchProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
                 return body().contains("Camel").matches(exchange);
             }
         };
 
-        mockCondition = new Condition() {
-            public boolean match(Exchange exchange, ProcessorDefinition definition) {
+        mockCondition = new ConditionSupport() {
+            public boolean matchProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
                 // match when sending to mocks
                 if (definition instanceof ToDefinition) {
                     ToDefinition to = (ToDefinition) definition;
@@ -67,6 +76,13 @@ public class DebugTest extends ContextTe
                 return false;
             }
         };
+
+        doneCondition = new ConditionSupport() {
+            @Override
+            public boolean matchEvent(Exchange exchange, EventObject event) {
+                return event instanceof ExchangeCompletedEvent;
+            }
+        };
     }
 
     public void testDebug() throws Exception {
@@ -84,6 +100,21 @@ public class DebugTest extends ContextTe
         assertEquals("Breakpoint at To[mock:result] with body: Hello Camel", logs.get(1));
     }
 
+    public void testDebugEvent() throws Exception {
+        context.getDebugger().addBreakpoint(breakpoint, doneCondition);
+
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hello World", "Hello Camel");
+
+        template.sendBody("direct:start", "Hello World");
+        template.sendBody("direct:start", "Hello Camel");
+
+        assertMockEndpointsSatisfied();
+
+        assertEquals(2, logs.size());
+        assertEquals("Breakpoint event ExchangeCompletedEvent with body: Hello World", logs.get(0));
+        assertEquals("Breakpoint event ExchangeCompletedEvent with body: Hello Camel", logs.get(1));
+    }
+
     public void testDebugSuspended() throws Exception {
         context.getDebugger().addBreakpoint(breakpoint, mockCondition, camelCondition);