You are viewing a plain text version of this content. The canonical link for it is here.
Posted to cvs@cocoon.apache.org by vg...@apache.org on 2005/03/11 21:47:18 UTC

svn commit: r157153 - in cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap: ErrorHandlerHelper.java PipelineNode.java PipelinesNode.java PipelinesNodeBuilder.java

Author: vgritsenko
Date: Fri Mar 11 12:47:16 2005
New Revision: 157153

URL: http://svn.apache.org/viewcvs?view=rev&rev=157153
Log:
Refactor sitemap error handling:
 * Encapsulate error handling details into ErrorHandlerHelper
 * Always log exception into handled-erorrs log, both in map:pipeline and map:pipelines
 * Error handling in map:pipelines gets same logic as in map:pipeline


Modified:
    cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ErrorHandlerHelper.java
    cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java
    cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNode.java
    cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java

Modified: cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ErrorHandlerHelper.java
URL: http://svn.apache.org/viewcvs/cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ErrorHandlerHelper.java?view=diff&r1=157152&r2=157153
==============================================================================
--- cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ErrorHandlerHelper.java (original)
+++ cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ErrorHandlerHelper.java Fri Mar 11 12:47:16 2005
@@ -1,12 +1,12 @@
 /*
- * Copyright 1999-2004 The Apache Software Foundation.
- * 
+ * Copyright 1999-2005 The Apache Software Foundation.
+ *
  * 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.
@@ -15,12 +15,14 @@
  */
 package org.apache.cocoon.components.treeprocessor.sitemap;
 
-import java.util.Map;
-
+import org.apache.avalon.framework.component.ComponentException;
 import org.apache.avalon.framework.component.ComponentManager;
 import org.apache.avalon.framework.component.Composable;
 import org.apache.avalon.framework.logger.AbstractLogEnabled;
+import org.apache.avalon.framework.logger.Logger;
+
 import org.apache.cocoon.Constants;
+import org.apache.cocoon.ResourceNotFoundException;
 import org.apache.cocoon.components.notification.Notifying;
 import org.apache.cocoon.components.notification.NotifyingBuilder;
 import org.apache.cocoon.components.treeprocessor.InvokeContext;
@@ -28,6 +30,9 @@
 import org.apache.cocoon.environment.Environment;
 import org.apache.cocoon.environment.ObjectModelHelper;
 
+import java.io.IOException;
+import java.util.Map;
+
 /**
  * Helps to call error handlers from PipelineNode and PipelinesNode.
  *
@@ -35,72 +40,128 @@
  * @author <a href="mailto:bluetkemeier@s-und-n.de">Bj&ouml;rn L&uuml;tkemeier</a>
  * @version CVS $Id$
  */
-public class ErrorHandlerHelper extends AbstractLogEnabled implements Composable {
+public class ErrorHandlerHelper extends AbstractLogEnabled
+                                implements Composable {
 
     private ComponentManager manager;
 
     /**
+     * Logger for handled errors
+     */
+    protected Logger handledErrorsLogger;
+
+    /**
+     * Error handling node for the ResourceNotFoundException
+     */
+    private ProcessingNode error404;
+
+    /**
+     * Error handling node for all other exceptions
+     */
+    private ProcessingNode error500;
+
+
+    public void enableLogging(Logger logger) {
+        super.enableLogging(logger);
+        this.handledErrorsLogger = logger.getChildLogger("handled-errors");
+    }
+
+    /**
      * The component manager is used to create notifying builders.
      */
     public void compose(ComponentManager manager) {
         this.manager = manager;
     }
 
-    public boolean invokeErrorHandler(ProcessingNode node, 
-                                      Exception ex, 
+    void setHandledErrorsLogger(Logger logger) {
+        this.handledErrorsLogger = logger;
+    }
+
+    void set404Handler(ProcessingNode node) {
+        this.error404 = node;
+    }
+
+    void set500Handler(ProcessingNode node) {
+        this.error500 = node;
+    }
+
+    public boolean invokeErrorHandler(Exception ex,
+                                      Environment env,
+                                      InvokeContext context)
+    throws Exception {
+        if (!env.isExternal() && !env.isInternalRedirect()) {
+            // Propagate exception on internal requests
+            throw ex;
+        } else if (error404 != null && ex instanceof ResourceNotFoundException) {
+            // Invoke 404-specific handler
+            return invokeErrorHandler(error404, ex, env, context);
+        } else if (error500 != null) {
+            // Invoke global handler
+            return invokeErrorHandler(error500, ex, env, context);
+        }
+
+        // No handler : propagate
+        throw ex;
+    }
+
+    public boolean invokeErrorHandler(ProcessingNode node,
+                                      Exception ex,
                                       Environment env,
-                                      InvokeContext originalContext)
+                                      InvokeContext context)
     throws Exception {
-		Map objectModel = env.getObjectModel();
-  	
-        InvokeContext errorContext = null;
-		boolean nodeSuccessful = false;
-		
+        this.handledErrorsLogger.error(ex.getMessage(), ex);
+
         try {
-        	if (objectModel.get(Constants.NOTIFYING_OBJECT) == null) {
-				// error has not been processed by another handler before
-				
-	            // Try to reset the response to avoid mixing already produced output
-	            // and error page.
-	            env.tryResetResponse();
-	
-	            // Create a Notifying
-	            NotifyingBuilder notifyingBuilder= (NotifyingBuilder)this.manager.lookup(NotifyingBuilder.ROLE);
-	            Notifying currentNotifying = null;
-	            try {
-	                currentNotifying = notifyingBuilder.build(this, ex);
-	            } finally {
-	                this.manager.release(notifyingBuilder);
-	            }
-	
-	            // Add it to the object model
-	            objectModel.put(Constants.NOTIFYING_OBJECT, currentNotifying);
-	            
-	            // Also add the exception
-	            objectModel.put(ObjectModelHelper.THROWABLE_OBJECT, ex);
-        	}
-
-			// Build a new context
-			errorContext = new InvokeContext();
-			errorContext.enableLogging(getLogger());
-			errorContext.compose(this.manager);
-			errorContext.setRedirector(originalContext.getRedirector());
-            
-			nodeSuccessful = node.invoke(env, errorContext);
-        } catch (Exception subEx) {
-            getLogger().error("An exception occured while handling errors at " + node.getLocation(), subEx);
-            // Rethrow it : it will either be handled by the parent sitemap or by the environment (e.g. Cocoon servlet)
-            throw subEx;
-        } finally {
-            if (errorContext != null) {
+            prepare(env, context, ex);
+
+            // Create error context
+            InvokeContext errorContext = new InvokeContext(context.isBuildingPipelineOnly());
+            errorContext.enableLogging(getLogger());
+            errorContext.setRedirector(context.getRedirector());
+            errorContext.compose(this.manager);
+            try {
+                // Process error handling node
+                if (node.invoke(env, errorContext)) {
+                    // Exception was handled.
+                    return true;
+                }
+            } finally {
                 errorContext.dispose();
             }
+        } catch (Exception e) {
+            getLogger().error("An exception occured while handling errors at " + node.getLocation(), e);
+            // Rethrow it: It will either be handled by the parent sitemap or by the environment (e.g. Cocoon servlet)
+            throw e;
         }
-        
-        if (nodeSuccessful) {
-            return true;
-        }
+
+        // Exception was not handled in this error handler, propagate.
         throw ex;
     }
-}
 
+    private void prepare(Environment env, InvokeContext context, Exception ex)
+    throws IOException, ComponentException {
+        Map objectModel = env.getObjectModel();
+        if (objectModel.get(Constants.NOTIFYING_OBJECT) == null) {
+            // error has not been processed by another handler before
+
+            // Try to reset the response to avoid mixing already produced output
+            // and error page.
+            env.tryResetResponse();
+
+            // Create a Notifying
+            NotifyingBuilder notifyingBuilder = (NotifyingBuilder) this.manager.lookup(NotifyingBuilder.ROLE);
+            Notifying currentNotifying = null;
+            try {
+                currentNotifying = notifyingBuilder.build(this, ex);
+            } finally {
+                this.manager.release(notifyingBuilder);
+            }
+
+            // Add it to the object model
+            objectModel.put(Constants.NOTIFYING_OBJECT, currentNotifying);
+
+            // Also add the exception
+            objectModel.put(ObjectModelHelper.THROWABLE_OBJECT, ex);
+        }
+    }
+}

Modified: cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java
URL: http://svn.apache.org/viewcvs/cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java?view=diff&r1=157152&r2=157153
==============================================================================
--- cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java (original)
+++ cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java Fri Mar 11 12:47:16 2005
@@ -1,12 +1,12 @@
 /*
- * Copyright 1999-2004 The Apache Software Foundation.
- * 
+ * Copyright 1999-2005 The Apache Software Foundation.
+ *
  * 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.
@@ -15,20 +15,20 @@
  */
 package org.apache.cocoon.components.treeprocessor.sitemap;
 
-import java.util.Map;
-
 import org.apache.avalon.framework.component.ComponentManager;
 import org.apache.avalon.framework.component.Composable;
 import org.apache.avalon.framework.logger.Logger;
+
 import org.apache.cocoon.ConnectionResetException;
 import org.apache.cocoon.ResourceNotFoundException;
 import org.apache.cocoon.components.treeprocessor.AbstractParentProcessingNode;
 import org.apache.cocoon.components.treeprocessor.InvokeContext;
 import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
 import org.apache.cocoon.components.treeprocessor.ProcessingNode;
-import org.apache.cocoon.components.treeprocessor.sitemap.MountNode;
 import org.apache.cocoon.environment.Environment;
 
+import java.util.Map;
+
 /**
  * Handles &lt;map:pipeline&gt;
  *
@@ -37,34 +37,27 @@
  * @author <a href="mailto:sylvain@apache.org">Sylvain Wallez</a>
  * @author <a href="mailto:gianugo@apache.org">Gianugo Rabellino</a>
  * @author <a href="mailto:cziegeler@apache.org">Carsten Ziegeler</a>
- * @version CVS $Id$
+ * @version $Id$
  */
-public class PipelineNode
-        extends AbstractParentProcessingNode
-        implements Composable, ParameterizableProcessingNode {
+public class PipelineNode extends AbstractParentProcessingNode
+                          implements Composable, ParameterizableProcessingNode {
 
-    // TODO : handle a 'fail-hard' environment attribute
-    // can be useful to stop off-line generation when there's an error
+    // TODO: handle a 'fail-hard' environment attribute
+    //       can be useful to stop off-line generation when there's an error
 
     private ProcessingNode[] children;
 
-    private ProcessingNode error404;
-
-    private ProcessingNode error500;
+    private ErrorHandlerHelper errorHandlerHelper;
 
-    private ErrorHandlerHelper errorHandlerHelper = new ErrorHandlerHelper();
-
-    protected Logger handledErrorsLogger;
-
-    private boolean internalOnly = false;
+    private boolean internalOnly;
 
     /** Is it the last <pipeline> in the enclosing <pipelines> ? */
-    private boolean isLast = false;
+    private boolean isLast;
 
     /** The component name of the processing pipeline */
     protected String processingPipeline;
 
-    /** Optional Sitemap parameters */
+    /** Optional sitemap parameters */
     protected Map parameters;
 
     /**
@@ -73,6 +66,7 @@
      */
     public PipelineNode(String name) {
         this.processingPipeline = name;
+        this.errorHandlerHelper = new ErrorHandlerHelper();
     }
 
     /**
@@ -85,7 +79,6 @@
     public void enableLogging(Logger logger) {
         super.enableLogging(logger);
         this.errorHandlerHelper.enableLogging(logger);
-        handledErrorsLogger = logger.getChildLogger("handled-errors");
     }
 
     public void setChildren(ProcessingNode[] nodes) {
@@ -101,11 +94,11 @@
     }
 
     public void set404Handler(ProcessingNode node) {
-        this.error404 = node;
+        this.errorHandlerHelper.set404Handler(node);
     }
 
     public void set500Handler(ProcessingNode node) {
-        this.error500 = node;
+        this.errorHandlerHelper.set500Handler(node);
     }
 
     public void setInternalOnly(boolean internalOnly) {
@@ -113,56 +106,38 @@
     }
 
     public final boolean invoke(Environment env, InvokeContext context)
-            throws Exception {
-
+    throws Exception {
         boolean passThrough;
-
         Object passThroughRaw = env.getAttribute(MountNode.COCOON_PASS_THROUGH);
-       
-        if(passThroughRaw == null){
-            //use legacy default value
+        if (passThroughRaw == null) {
+            // Use default value
             passThrough = false;
-        }else{
-            passThrough = ((Boolean)passThroughRaw).booleanValue() ;
+        } else {
+            passThrough = ((Boolean) passThroughRaw).booleanValue();
         }
-        
-        boolean externalRequest = env.isExternal();
 
         // Always fail on external request if pipeline is internal only.
-        if (this.internalOnly && externalRequest) {
+        if (this.internalOnly && env.isExternal()) {
             return false;
         }
-        context.inform(this.processingPipeline, this.parameters,
-                env.getObjectModel());
+
+        context.inform(this.processingPipeline, this.parameters, env.getObjectModel());
         try {
             if (invokeNodes(children, env, context)) {
                 return true;
             } else if (!this.isLast || passThrough) {
                 return false;
-            } else {
-                throw new ResourceNotFoundException(
-                        "No pipeline matched request: " + env.getURIPrefix()
-                        + env.getURI());
             }
-        } catch (ConnectionResetException cre) {
+
+            throw new ResourceNotFoundException("No pipeline matched request: " +
+                                                env.getURIPrefix() + env.getURI());
+
+        } catch (ConnectionResetException e) {
             // Will be reported by CocoonServlet, rethrowing
-            throw cre;
-        } catch (Exception ex) {
-            if (!externalRequest && !env.isInternalRedirect()) {
-                // Propagate exception on internal requests
-                throw ex;
-            } else if (error404 != null && ex instanceof ResourceNotFoundException) {
-                // Invoke 404-specific handler
-                handledErrorsLogger.error(ex.getMessage(), ex);
-                return errorHandlerHelper.invokeErrorHandler(error404, ex, env, context);
-            } else if (error500 != null) {
-                // Invoke global handler
-                handledErrorsLogger.error(ex.getMessage(), ex);
-                return errorHandlerHelper.invokeErrorHandler(error500, ex, env, context);
-            } else {
-                // No handler : propagate
-                throw ex;
-            }
+            throw e;
+        } catch (Exception e) {
+            // Invoke error handler
+            return this.errorHandlerHelper.invokeErrorHandler(e, env, context);
         }
     }
 }

Modified: cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNode.java
URL: http://svn.apache.org/viewcvs/cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNode.java?view=diff&r1=157152&r2=157153
==============================================================================
--- cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNode.java (original)
+++ cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNode.java Fri Mar 11 12:47:16 2005
@@ -1,12 +1,12 @@
 /*
  * Copyright 1999-2004 The Apache Software Foundation.
- * 
+ *
  * 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.
@@ -19,6 +19,8 @@
 import org.apache.avalon.framework.component.ComponentManager;
 import org.apache.avalon.framework.component.Composable;
 import org.apache.avalon.framework.logger.Logger;
+
+import org.apache.cocoon.ConnectionResetException;
 import org.apache.cocoon.components.treeprocessor.InvokeContext;
 import org.apache.cocoon.components.treeprocessor.ProcessingNode;
 import org.apache.cocoon.components.treeprocessor.SimpleParentProcessingNode;
@@ -31,17 +33,19 @@
  * @author <a href="mailto:bluetkemeier@s-und-n.de">Bj&ouml;rn L&uuml;tkemeier</a>
  * @author <a href="mailto:sylvain@apache.org">Sylvain Wallez</a>
  * @author <a href="mailto:cziegeler@apache.org">Carsten Ziegeler</a>
- * @version CVS $Id$
+ * @version $Id$
  */
-
 public final class PipelinesNode extends SimpleParentProcessingNode
-  implements Composable, Disposable {
+                                 implements Composable, Disposable {
 
     private ComponentManager manager;
-    
-    private ErrorHandlerHelper errorHandlerHelper = new ErrorHandlerHelper();
 
-    private ProcessingNode errorHandler;
+    private ErrorHandlerHelper errorHandlerHelper;
+
+
+    public PipelinesNode() {
+        this.errorHandlerHelper = new ErrorHandlerHelper();
+    }
 
     /**
      * Keep the component manager used everywhere in the tree so that we can
@@ -58,13 +62,12 @@
     }
 
     public void setErrorHandler(ProcessingNode node) {
-        this.errorHandler = node;
+        this.errorHandlerHelper.set500Handler(node);
     }
-    
+
     public void setChildren(ProcessingNode[] nodes) {
         // Mark the last pipeline so that it can throw a ResourceNotFoundException
-        ((PipelineNode)nodes[nodes.length - 1]).setLast(true);
-
+        ((PipelineNode) nodes[nodes.length - 1]).setLast(true);
         super.setChildren(nodes);
     }
 
@@ -75,25 +78,25 @@
      */
     public final boolean invoke(Environment env, InvokeContext context)
     throws Exception {
-    
-        // Perform any common invoke functionality 
+
+        // Perform any common invoke functionality
         super.invoke(env, context);
 
         // Recompose context (and pipelines) to the local component manager
         context.recompose(this.manager);
 
         try {
-            // FIXME : is there any useful information that can be passed as top-level parameters,
-            //         such as the URI of the mount point ?
+            // FIXME: Is there any useful information that can be passed as top-level parameters,
+            //        such as the URI of the mount point ?
 
             return invokeNodes(this.children, env, context);
+
+        } catch (ConnectionResetException e) {
+            // Will be reported by CocoonServlet, rethrowing
+            throw e;
         } catch (Exception ex) {
-            if (this.errorHandler != null) {
-                // Invoke pipelines handler
-                return this.errorHandlerHelper.invokeErrorHandler(this.errorHandler, ex, env, context);
-            } 
-            // No handler : propagate
-            throw ex;
+            // Invoke pipelines handler
+            return this.errorHandlerHelper.invokeErrorHandler(ex, env, context);
         }
     }
 
@@ -102,7 +105,7 @@
      */
     public void dispose() {
         if (this.manager instanceof Disposable) {
-            ((Disposable)this.manager).dispose();
+            ((Disposable) this.manager).dispose();
         }
         this.manager = null;
     }

Modified: cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java
URL: http://svn.apache.org/viewcvs/cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java?view=diff&r1=157152&r2=157153
==============================================================================
--- cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java (original)
+++ cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java Fri Mar 11 12:47:16 2005
@@ -1,12 +1,12 @@
 /*
- * Copyright 1999-2004 The Apache Software Foundation.
- * 
+ * Copyright 1999-2005 The Apache Software Foundation.
+ *
  * 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.
@@ -32,28 +32,27 @@
  * @author <a href="mailto:bluetkemeier@s-und-n.de">Bj&ouml;rn L&uuml;tkemeier</a>
  * @author <a href="mailto:sylvain@apache.org">Sylvain Wallez</a>
  * @author <a href="mailto:cziegeler@apache.org">Carsten Ziegeler</a>
- * @version CVS $Id: PipelinesNodeBuilder.java,v 1.5 2004/03/05 13:02:52 bdelacretaz Exp $
+ * @version $Id$
  */
-
 public class PipelinesNodeBuilder extends ContainerNodeBuilder implements ThreadSafe {
 
     public ProcessingNode buildNode(Configuration config)
     throws Exception {
-        // check for component configurations
+        // Check for component configurations
         Configuration child = config.getChild("component-configurations", false);
         if (child != null) {
-            this.checkNamespace(child);
+            checkNamespace(child);
             this.treeBuilder.getProcessor().setComponentConfigurations(child);
         }
+
         PipelinesNode node = new PipelinesNode();
         this.treeBuilder.setupNode(node, config);
 
 		Configuration[] childConfigs = config.getChildren();
 		List children = new ArrayList();
 		HandleErrorsNode handler = null;
-		
-		for (int i = 0; i < childConfigs.length; i++) {
 
+		for (int i = 0; i < childConfigs.length; i++) {
 			Configuration childConfig = childConfigs[i];
 			if (isChild(childConfig)) {
 
@@ -66,6 +65,7 @@
 				}
 			}
 		}
+
         if (children.size() == 0) {
             String msg = "There must be at least one pipeline at " + config.getLocation();
             throw new ConfigurationException(msg);



Re: svn commit: r157153 - in cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap: ErrorHandlerHelper.java PipelineNode.java PipelinesNode.java PipelinesNodeBuilder.java

Posted by Vadim Gritsenko <va...@reverycodes.com>.
Sylvain Wallez wrote:
> Vadim Gritsenko wrote:
> 
>> Ralph Goers wrote:
>>
>>> Does this fix mean that handle-errors now works on internal pipelines 
>>> as well?
>>
>>
>> No, not yet, I need couple of more days. But it means that:
>>
>>   <map:pipelines>
>>     <map:pipeline>
>>       ...
>>       <map:handle-errors/> (1)
>>     </map:pipeline>
>>     <map:handle-errors/> (2)
>>   </map:pipelines>
>>
>> (2) is now behaving more like (1). Before, I noticed it did not log an 
>> error into handled-error log, did not handle ConnectionResetException, 
>> etc.
> 
> 
> Sorry, I may have missed something as I never heard of (2). When is it 
> triggered?

If you remember, exception "traverses" error handlers up the invocation tree 
till it finds the handler which can process exception. So, (2) will be triggered 
if (1) is missing or did not process an exception. Behaviour is very similar to 
java exception catch block processing.

I see that this feature was introduced before 2.1m1 release [1]. See also [2], 
[3] for samples showing this off.


> Also, when no match is found, the <handle-errors> of the last 
> <map:pipeline> was used to handle the ResourceNotFound. Is this still 
> the case?

Yes.

Vadim

[1] 
http://cvs.apache.org/viewcvs.cgi/cocoon-2.1/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelinesNodeBuilder.java?r1=1.1&r2=1.2&diff_format=h
[2] 
http://svn.apache.org/repos/asf/cocoon/branches/BRANCH_2_1_X/src/webapp/samples/errorhandling/exception/sitemap.xmap
[3] 
http://svn.apache.org/repos/asf/cocoon/branches/BRANCH_2_1_X/src/webapp/samples/errorhandling/sitemap.xmap

Re: svn commit: r157153 - in cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap: ErrorHandlerHelper.java PipelineNode.java PipelinesNode.java PipelinesNodeBuilder.java

Posted by Sylvain Wallez <sy...@apache.org>.
Vadim Gritsenko wrote:

> Ralph Goers wrote:
>
>> Does this fix mean that handle-errors now works on internal pipelines 
>> as well?
>
>
> No, not yet, I need couple of more days. But it means that:
>
>   <map:pipelines>
>     <map:pipeline>
>       ...
>       <map:handle-errors/> (1)
>     </map:pipeline>
>     <map:handle-errors/> (2)
>   </map:pipelines>
>
> (2) is now behaving more like (1). Before, I noticed it did not log an 
> error into handled-error log, did not handle ConnectionResetException, 
> etc.


Sorry, I may have missed something as I never heard of (2). When is it 
triggered?

Also, when no match is found, the <handle-errors> of the last 
<map:pipeline> was used to handle the ResourceNotFound. Is this still 
the case?

Sylvain

-- 
Sylvain Wallez                                  Anyware Technologies
http://www.apache.org/~sylvain           http://www.anyware-tech.com
{ XML, Java, Cocoon, OpenSource }*{ Training, Consulting, Projects }


Re: svn commit: r157153 - in cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap: ErrorHandlerHelper.java PipelineNode.java PipelinesNode.java PipelinesNodeBuilder.java

Posted by Vadim Gritsenko <va...@reverycodes.com>.
Ralph Goers wrote:
> Does this fix mean that handle-errors now works on internal pipelines as 
> well?

No, not yet, I need couple of more days. But it means that:

   <map:pipelines>
     <map:pipeline>
       ...
       <map:handle-errors/> (1)
     </map:pipeline>
     <map:handle-errors/> (2)
   </map:pipelines>

(2) is now behaving more like (1). Before, I noticed it did not log an error 
into handled-error log, did not handle ConnectionResetException, etc.

Vadim


> Ralph
> 
> vgritsenko@apache.org wrote:
> 
>> Author: vgritsenko
>> Date: Fri Mar 11 12:47:16 2005
>> New Revision: 157153
>>
>> URL: http://svn.apache.org/viewcvs?view=rev&rev=157153
>> Log:
>> Refactor sitemap error handling:
>> * Encapsulate error handling details into ErrorHandlerHelper
>> * Always log exception into handled-erorrs log, both in map:pipeline 
>> and map:pipelines
>> * Error handling in map:pipelines gets same logic as in map:pipeline

Re: svn commit: r157153 - in cocoon/branches/BRANCH_2_1_X/src/java/org/apache/cocoon/components/treeprocessor/sitemap: ErrorHandlerHelper.java PipelineNode.java PipelinesNode.java PipelinesNodeBuilder.java

Posted by Ralph Goers <Ra...@dslextreme.com>.
Does this fix mean that handle-errors now works on internal pipelines as 
well?

Ralph

vgritsenko@apache.org wrote:

>Author: vgritsenko
>Date: Fri Mar 11 12:47:16 2005
>New Revision: 157153
>
>URL: http://svn.apache.org/viewcvs?view=rev&rev=157153
>Log:
>Refactor sitemap error handling:
> * Encapsulate error handling details into ErrorHandlerHelper
> * Always log exception into handled-erorrs log, both in map:pipeline and map:pipelines
> * Error handling in map:pipelines gets same logic as in map:pipeline
>
>  
>