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 10:29:21 UTC

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

Author: davsclaus
Date: Thu Jul  8 08:29:21 2010
New Revision: 961627

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

Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/BreakpointSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultDebugger.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/Debugger.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupportJmxCleanup.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java?rev=961627&r1=961626&r2=961627&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/Exchange.java Thu Jul  8 08:29:21 2010
@@ -40,32 +40,28 @@ public interface Exchange {
     String AGGREGATION_STRATEGY             = "CamelAggregationStrategy";
     String ASYNC_WAIT                       = "CamelAsyncWait";
 
-    String BATCH_INDEX    = "CamelBatchIndex";
-    String BATCH_SIZE     = "CamelBatchSize";
-    String BATCH_COMPLETE = "CamelBatchComplete";
-
+    String BATCH_INDEX                = "CamelBatchIndex";
+    String BATCH_SIZE                 = "CamelBatchSize";
+    String BATCH_COMPLETE             = "CamelBatchComplete";
     String BEAN_METHOD_NAME           = "CamelBeanMethodName";
     String BEAN_MULTI_PARAMETER_ARRAY = "CamelBeanMultiParameterArray";
-
-    String BINDING = "CamelBinding";
+    String BINDING                    = "CamelBinding";
 
     String CHARSET_NAME     = "CamelCharsetName";
     String CONTENT_ENCODING = "Content-Encoding";
     String CONTENT_TYPE     = "Content-Type";
     String CORRELATION_ID   = "CamelCorrelationId";
 
-    String DATASET_INDEX = "CamelDataSetIndex";
-    String DEFAULT_CHARSET_PROPERTY = "org.apache.camel.default.charset";
-    
+    String DATASET_INDEX             = "CamelDataSetIndex";
+    String DEFAULT_CHARSET_PROPERTY  = "org.apache.camel.default.charset";
     String DISABLE_HTTP_STREAM_CACHE = "CamelDisableHttpStreamCache";
 
     String EXCEPTION_CAUGHT     = "CamelExceptionCaught";
     String ERRORHANDLER_HANDLED = "CamelErrorHandlerHandled";
+
     String FAILURE_HANDLED      = "CamelFailureHandled";
     String FAILURE_ENDPOINT     = "CamelFailureEndpoint";
-    
     String FILTER_NON_XML_CHARS = "CamelFilterNonXmlChars";
-
     String FILE_LOCAL_WORK_PATH = "CamelFileLocalWorkPath";
     String FILE_NAME            = "CamelFileName";
     String FILE_NAME_ONLY       = "CamelFileNameOnly";
@@ -85,11 +81,10 @@ public interface Exchange {
     String HTTP_URI                = "CamelHttpUri";
     String HTTP_URL                = "CamelHttpUrl";
     String HTTP_CHUNKED            = "CamelHttpChunked";
-    String HTTP_SERVLET_REQUEST   = "CamelHttpServletRequest";
-    String HTTP_SERVLET_RESPONSE  = "CamelHttpServletResponse";
+    String HTTP_SERVLET_REQUEST    = "CamelHttpServletRequest";
+    String HTTP_SERVLET_RESPONSE   = "CamelHttpServletResponse";
 
     String INTERCEPTED_ENDPOINT = "CamelInterceptedEndpoint";
-    String TO_ENDPOINT          = "CamelToEndpoint";
 
     String LOG_DEBUG_BODY_MAX_CHARS = "CamelLogDebugBodyMaxChars";
     String LOG_DEBUG_BODY_STREAMS   = "CamelLogDebugStreams";
@@ -102,29 +97,29 @@ public interface Exchange {
 
     String ON_COMPLETION = "CamelOnCompletion";
 
-    String ROUTE_STOP           = "CamelRouteStop";
     String REDELIVERED          = "CamelRedelivered";
     String REDELIVERY_COUNTER   = "CamelRedeliveryCounter";
     String REDELIVERY_EXHAUSTED = "CamelRedeliveryExhausted";
     String ROLLBACK_ONLY        = "CamelRollbackOnly";
     String ROLLBACK_ONLY_LAST   = "CamelRollbackOnlyLast";
-    
+    String ROUTE_STOP           = "CamelRouteStop";
+
     String SOAP_ACTION        = "CamelSoapAction";
     String SKIP_GZIP_ENCODING = "CamelSkipGzipEncoding";
     String SPLIT_INDEX        = "CamelSplitIndex";
     String SPLIT_COMPLETE     = "CamelSplitComplete";
     String SPLIT_SIZE         = "CamelSplitSize";
 
-    String TRANSFER_ENCODING     = "Transfer-Encoding";
+    String TIMER_FIRED_TIME      = "CamelTimerFiredTime";
+    String TIMER_NAME            = "CamelTimerName";
+    String TIMER_PERIOD          = "CamelTimerPeriod";
+    String TIMER_TIME            = "CamelTimerTime";
+    String TO_ENDPOINT           = "CamelToEndpoint";
     String TRACE_EVENT           = "CamelTraceEvent";
     String TRACE_EVENT_NODE_ID   = "CamelTraceEventNodeId";
     String TRACE_EVENT_TIMESTAMP = "CamelTraceEventTimestamp";
     String TRACE_EVENT_EXCHANGE  = "CamelTraceEventExchange";
-
-    String TIMER_FIRED_TIME = "CamelTimerFiredTime";
-    String TIMER_NAME       = "CamelTimerName";
-    String TIMER_PERIOD     = "CamelTimerPeriod";
-    String TIMER_TIME       = "CamelTimerTime";
+    String TRANSFER_ENCODING     = "Transfer-Encoding";
 
     String XSLT_FILE_NAME = "CamelXsltFileName";
 

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=961627&r1=961626&r2=961627&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 08:29:21 2010
@@ -50,7 +50,7 @@ public abstract class BreakpointSupport 
         // noop
     }
 
-    public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+    public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken) {
         // noop
     }
 

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=961627&r1=961626&r2=961627&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 08:29:21 2010
@@ -139,7 +139,7 @@ public class DefaultDebugger implements 
         return match;
     }
 
-    public boolean afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+    public boolean afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken) {
         boolean match = false;
 
         // does any of the breakpoints apply?
@@ -148,7 +148,7 @@ public class DefaultDebugger implements 
             if (Breakpoint.State.Active.equals(breakpoint.getBreakpoint().getState())) {
                 if (matchConditions(exchange, processor, definition, breakpoint)) {
                     match = true;
-                    onAfterProcess(exchange, processor, definition, breakpoint.getBreakpoint());
+                    onAfterProcess(exchange, processor, definition, timeTaken, breakpoint.getBreakpoint());
                 }
             }
         }
@@ -181,9 +181,9 @@ public class DefaultDebugger implements 
         }
     }
 
-    protected void onAfterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, Breakpoint breakpoint) {
+    protected void onAfterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken, Breakpoint breakpoint) {
         try {
-            breakpoint.afterProcess(exchange, processor, definition);
+            breakpoint.afterProcess(exchange, processor, definition, timeTaken);
         } catch (Throwable e) {
             LOG.warn("Exception occurred in breakpoint: " + breakpoint + ". This exception will be ignored.", e);
         }

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=961627&r1=961626&r2=961627&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 08:29:21 2010
@@ -24,6 +24,7 @@ import org.apache.camel.model.ProcessorD
 import org.apache.camel.processor.DelegateAsyncProcessor;
 import org.apache.camel.spi.Debugger;
 import org.apache.camel.spi.InterceptStrategy;
+import org.apache.camel.util.StopWatch;
 
 /**
  * A debug interceptor to notify {@link Debugger} with {@link Exchange}s being processed.
@@ -44,10 +45,13 @@ public class Debug implements InterceptS
             @Override
             public boolean process(final Exchange exchange, final AsyncCallback callback) {
                 debugger.beforeProcess(exchange, target, definition);
+                final StopWatch watch = new StopWatch();
 
                 return super.process(exchange, new AsyncCallback() {
                     public void done(boolean doneSync) {
-                        debugger.afterProcess(exchange, processor, definition);
+                        long diff = watch.stop();
+                        debugger.afterProcess(exchange, processor, definition, diff);
+
                         // must notify original callback
                         callback.done(doneSync);
                     }

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=961627&r1=961626&r2=961627&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 08:29:21 2010
@@ -74,8 +74,9 @@ public interface Breakpoint {
      * @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
+     * @param timeTaken  time in millis it took to process the {@link Exchange} - time spend in breakpoint callbacks may affect this time
      */
-    void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition);
+    void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken);
 
     /**
      * Callback invoked when the breakpoint was hit and any of the {@link Exchange} {@link EventObject event}s occurred.

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=961627&r1=961626&r2=961627&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 08:29:21 2010
@@ -90,9 +90,10 @@ public interface Debugger extends Servic
      * @param exchange   the exchange
      * @param processor  the {@link Processor} which was processed
      * @param definition the definition of the processor
+     * @param timeTaken  time in millis it took to process the {@link Exchange} - time spend in breakpoint callbacks may affect this time
      * @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);
+    boolean afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken);
 
     /**
      * Callback invoked when an {@link Exchange} is being processed which allows implementators

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupportJmxCleanup.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupportJmxCleanup.java?rev=961627&r1=961626&r2=961627&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupportJmxCleanup.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupportJmxCleanup.java Thu Jul  8 08:29:21 2010
@@ -14,12 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.camel;
 
 import java.lang.management.ManagementFactory;
 import java.util.Set;
-
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
@@ -30,6 +28,10 @@ import org.apache.commons.logging.LogFac
 public final class TestSupportJmxCleanup {
     private static final transient Log LOG = LogFactory.getLog(TestSupportJmxCleanup.class);
 
+    private TestSupportJmxCleanup() {
+        // no instances
+    }
+
     public static void removeMBeans(String domain) throws Exception {
         MBeanServer mbsc =  ManagementFactory.getPlatformMBeanServer();
         Set<ObjectName> s = mbsc.queryNames(new ObjectName(getDomainName(domain) + ":*"), null);
@@ -55,7 +57,4 @@ public final class TestSupportJmxCleanup
         return domain == null ? DefaultManagementAgent.DEFAULT_DOMAIN : domain;
     }
 
-    private TestSupportJmxCleanup() {
-        // no instances
-    }
 }

Modified: 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=961627&r1=961626&r2=961627&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/interceptor/DebugExceptionBreakpointTest.java Thu Jul  8 08:29:21 2010
@@ -45,7 +45,7 @@ public class DebugExceptionBreakpointTes
 
         breakpoint = new BreakpointSupport() {
             @Override
-            public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition) {
+            public void afterProcess(Exchange exchange, Processor processor, ProcessorDefinition definition, long timeTaken) {
                 Exception e = exchange.getException();
                 logs.add("Breakpoint at " + definition.getShortName() + " caused by: " + e.getClass().getSimpleName() + "[" + e.getMessage() + "]");
             }