You are viewing a plain text version of this content. The canonical link for it is here.
Posted to cvs@cocoon.apache.org by sy...@apache.org on 2002/06/24 22:32:20 UTC

cvs commit: xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/variables NOPVariableResolver.java PreparedVariableResolver.java VariableResolver.java VariableResolverFactory.java

sylvain     2002/06/24 13:32:20

  Modified:    .        changes.xml
               src/java/org/apache/cocoon/components/treeprocessor
                        AbstractProcessingNodeBuilder.java
                        DefaultTreeBuilder.java InvokeContext.java
                        MapStackResolver.java
                        ParameterizableProcessingNode.java
               src/java/org/apache/cocoon/components/treeprocessor/sitemap
                        ActNodeBuilder.java ActSetNode.java
                        ActTypeNode.java ActionSetNode.java
                        ActionSetNodeBuilder.java AggregateNode.java
                        AggregateNodeBuilder.java CallFunctionNode.java
                        CallNode.java ContinueNode.java
                        ContinueNodeBuilder.java GenerateNode.java
                        GenerateNodeBuilder.java MatchNode.java
                        MatchNodeBuilder.java MountNode.java
                        MountNodeBuilder.java PipelineNode.java
                        PreparableMatchNode.java ReadNode.java
                        ReadNodeBuilder.java RedirectToNodeBuilder.java
                        RedirectToURINode.java ScriptNodeBuilder.java
                        SelectNode.java SelectNodeBuilder.java
                        SwitchSelectNode.java TransformNode.java
                        TransformNodeBuilder.java
  Added:       src/java/org/apache/cocoon/components/treeprocessor/variables
                        NOPVariableResolver.java
                        PreparedVariableResolver.java VariableResolver.java
                        VariableResolverFactory.java
  Log:
  Extend the sitemap variable substitution syntax using InputModules.
  
  Revision  Changes    Path
  1.195     +7 -1      xml-cocoon2/changes.xml
  
  Index: changes.xml
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/changes.xml,v
  retrieving revision 1.194
  retrieving revision 1.195
  diff -u -r1.194 -r1.195
  --- changes.xml	21 Jun 2002 14:15:13 -0000	1.194
  +++ changes.xml	24 Jun 2002 20:32:18 -0000	1.195
  @@ -39,6 +39,12 @@
    </devs>
   
    <release version="@version@" date="@date@">
  +  <action dev="SW" type="add">
  +   Extend the syntax of sitemap variable substitution : variables can have a prefix which
  +   indicates an InputModule where to get the value from. For example, the expression
  +   "{request-param:foo}" will evaluate to the value of the "foo" request parameter (provided
  +   of course that the "request-param" InputModule is correctly defined in cocoon.xconf).
  +  </action>
     <action dev="VG" type="fix" fixes-bug="9693" due-to="Vladimir Kralik" due-to-email="vladimir.kralik@asset.sk">
      Properly escape (national) characters included in element's attributes
      in XSP page.
  
  
  
  1.4       +17 -3     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/AbstractProcessingNodeBuilder.java
  
  Index: AbstractProcessingNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/AbstractProcessingNodeBuilder.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- AbstractProcessingNodeBuilder.java	2 May 2002 20:17:05 -0000	1.3
  +++ AbstractProcessingNodeBuilder.java	24 Jun 2002 20:32:18 -0000	1.4
  @@ -52,10 +52,16 @@
   
   import org.apache.avalon.framework.logger.AbstractLoggable;
   
  +import org.apache.avalon.framework.component.Composable;
  +import org.apache.avalon.framework.component.ComponentManager;
  +import org.apache.avalon.framework.component.ComponentException;
   import org.apache.avalon.framework.configuration.Configurable;
   import org.apache.avalon.framework.configuration.Configuration;
   import org.apache.avalon.framework.configuration.ConfigurationException;
   
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
  +
   import org.apache.cocoon.sitemap.PatternException;
   
   import java.util.Map;
  @@ -69,9 +75,15 @@
   
   
   public abstract class AbstractProcessingNodeBuilder extends AbstractLoggable
  -  implements ProcessingNodeBuilder {
  +  implements ProcessingNodeBuilder, Composable {
   
       protected TreeBuilder treeBuilder;
  +    
  +    protected ComponentManager manager;
  +
  +    public void compose(ComponentManager manager) throws ComponentException {
  +        this.manager = manager;
  +    }
   
       public void setBuilder(TreeBuilder treeBuilder) {
           this.treeBuilder = treeBuilder;
  @@ -106,7 +118,9 @@
                   String name = child.getAttribute("name");
                   String value = child.getAttribute("value");
                   try {
  -                    params.put(MapStackResolver.getResolver(name), MapStackResolver.getResolver(value));
  +                    params.put(
  +                        VariableResolverFactory.getResolver(name, this.manager),
  +                        VariableResolverFactory.getResolver(value, this.manager));
                   } catch(PatternException pe) {
                       String msg = "Invalid pattern '" + value + " at " + child.getLocation();
                       getLogger().error(msg, pe);
  
  
  
  1.10      +10 -2     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/DefaultTreeBuilder.java
  
  Index: DefaultTreeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/DefaultTreeBuilder.java,v
  retrieving revision 1.9
  retrieving revision 1.10
  diff -u -r1.9 -r1.10
  --- DefaultTreeBuilder.java	13 Jun 2002 11:25:48 -0000	1.9
  +++ DefaultTreeBuilder.java	24 Jun 2002 20:32:18 -0000	1.10
  @@ -82,6 +82,8 @@
   import org.apache.cocoon.components.ExtendedComponentSelector;
   import org.apache.cocoon.components.LifecycleHelper;
   import org.apache.cocoon.components.source.SourceUtil;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   import org.apache.cocoon.environment.SourceResolver;
   import org.apache.excalibur.source.Source;
   import org.apache.excalibur.source.SourceException;
  @@ -424,6 +426,9 @@
           // Calls to getRegisteredNode() are forbidden
           this.canGetNode = false;
   
  +        // Collect all disposable variable resolvers
  +        VariableResolverFactory.setDisposableCollector(this.disposableNodes);
  +
           ProcessingNode result = createTree(tree);
   
           // Calls to getRegisteredNode() are now allowed
  @@ -503,7 +508,9 @@
                   String name = child.getAttribute("name");
                   String value = child.getAttribute("value");
                   try {
  -                    params.put(MapStackResolver.getResolver(name), MapStackResolver.getResolver(value));
  +                    params.put(
  +                        VariableResolverFactory.getResolver(name, this.manager),
  +                        VariableResolverFactory.getResolver(value, this.manager));
                   } catch(PatternException pe) {
                       String msg = "Invalid pattern '" + value + "' at " + child.getLocation();
                       getLogger().error(msg, pe);
  @@ -570,6 +577,7 @@
   
           // Don't clear disposableNodes as they're used by the Processor
           this.disposableNodes = new ArrayList();
  +        VariableResolverFactory.setDisposableCollector(null);
   
           this.processor = null;
           this.manager = null;
  
  
  
  1.11      +10 -4     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/InvokeContext.java
  
  Index: InvokeContext.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/InvokeContext.java,v
  retrieving revision 1.10
  retrieving revision 1.11
  diff -u -r1.10 -r1.11
  --- InvokeContext.java	29 May 2002 09:12:04 -0000	1.10
  +++ InvokeContext.java	24 Jun 2002 20:32:18 -0000	1.11
  @@ -59,6 +59,7 @@
   import org.apache.avalon.framework.logger.Loggable;
   import org.apache.cocoon.components.CocoonComponentManager;
   import org.apache.cocoon.components.pipeline.ProcessingPipeline;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.environment.Environment;
   import org.apache.log.Logger;
   
  @@ -99,6 +100,9 @@
   
       /** The parameters for the processing pipeline */
       protected Map processingPipelineParameters;
  +    
  +    /** The object model used to resolve processingPipelineParameters */
  +    protected Map processingPipelineObjectModel;
   
       /** The ProcessingPipeline used */
       protected ProcessingPipeline processingPipeline;
  @@ -151,9 +155,11 @@
        * Informs the context about a new pipeline section
        */
       public void inform(String pipelineName,
  -                       Map    parameters) {
  +                       Map    parameters,
  +                       Map    objectModel) {
           this.processingPipelineName = pipelineName;
           this.processingPipelineParameters = parameters;
  +        this.processingPipelineObjectModel = objectModel;
       }
   
       /**
  @@ -169,8 +175,8 @@
               this.processingPipeline = (ProcessingPipeline)this.pipelineSelector.select(this.processingPipelineName);
               this.processingPipeline.recompose( this.pipelinesManager );
               this.processingPipeline.setup(
  -                  MapStackResolver.buildParameters(this.processingPipelineParameters,
  -                                                   this.getMapStack())
  +                  VariableResolver.buildParameters(this.processingPipelineParameters,
  +                                                   this.getMapStack(), this.processingPipelineObjectModel)
               );
               if (this.isInternalRequest) {
                   CocoonComponentManager.addComponentForAutomaticRelease(this.pipelineSelector,
  
  
  
  1.4       +2 -1      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/MapStackResolver.java
  
  Index: MapStackResolver.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/MapStackResolver.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- MapStackResolver.java	24 Apr 2002 08:49:52 -0000	1.3
  +++ MapStackResolver.java	24 Jun 2002 20:32:18 -0000	1.4
  @@ -61,6 +61,7 @@
    *
    * @author <a href="mailto:sylvain@apache.org">Sylvain Wallez</a>
    * @version CVS $Id$
  + * @deprecated use {@link org.apache.cocoon.treeprocessor.variables.VariableResolverFactory}
    */
   
   public abstract class MapStackResolver {
  
  
  
  1.2       +2 -2      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/ParameterizableProcessingNode.java
  
  Index: ParameterizableProcessingNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/ParameterizableProcessingNode.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ParameterizableProcessingNode.java	5 Mar 2002 08:26:22 -0000	1.1
  +++ ParameterizableProcessingNode.java	24 Jun 2002 20:32:18 -0000	1.2
  @@ -62,7 +62,7 @@
   public interface ParameterizableProcessingNode extends ProcessingNode {
   
       /**
  -     * Set the parameters of this node as a <code>Map</code> of <code>MapStackResolver</code>s
  +     * Set the parameters of this node as a <code>Map</code> of <code>VariableResolver</code>s
        * that will be resolved at process-time.
        */
       void setParameters(Map parameterMap);
  
  
  
  1.2       +7 -3      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActNodeBuilder.java
  
  Index: ActNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ActNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ ActNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -58,9 +58,10 @@
   import org.apache.cocoon.components.treeprocessor.AbstractParentProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
   
  @@ -85,7 +86,10 @@
               String source = config.getAttribute("src", null);
               String type = this.treeBuilder.getTypeForStatement(config, Action.ROLE + "Selector");
   
  -            ActTypeNode actTypeNode = new ActTypeNode(type, source);
  +            ActTypeNode actTypeNode = new ActTypeNode(
  +                type,
  +                VariableResolverFactory.getResolver(source, this.manager)
  +            );
               this.treeBuilder.setupNode(actTypeNode, config);
   
               actTypeNode.setChildren(buildChildNodes(config));
  
  
  
  1.4       +7 -4      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActSetNode.java
  
  Index: ActSetNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActSetNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- ActSetNode.java	29 Apr 2002 14:46:16 -0000	1.3
  +++ ActSetNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -64,7 +64,7 @@
   import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleParentProcessingNode;
  @@ -99,8 +99,11 @@
       public final boolean invoke(Environment env, InvokeContext context)
         throws Exception {
   
  -        List       mapStack = context.getMapStack();
  -        Parameters resolvedParams = MapStackResolver.buildParameters(this.parameters, mapStack);
  +        Parameters resolvedParams = VariableResolver.buildParameters(
  +            this.parameters,
  +            context.getMapStack(),
  +            env.getObjectModel()
  +        );
   
           Map result = this.actionSet.call(env, context, resolvedParams);
   
  
  
  
  1.4       +7 -7      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActTypeNode.java
  
  Index: ActTypeNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActTypeNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- ActTypeNode.java	22 Apr 2002 15:43:36 -0000	1.3
  +++ ActTypeNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -63,7 +63,7 @@
   import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
   
  @@ -85,16 +85,16 @@
       private Map parameters;
   
       /** The 'src' attribute */
  -    protected MapStackResolver source;
  +    protected VariableResolver source;
   
       /** Pre-selected action, if it's ThreadSafe */
       protected Action threadSafeAction;
   
       protected ComponentManager manager;
   
  -    public ActTypeNode(String name, String source) throws PatternException {
  +    public ActTypeNode(String name, VariableResolver source) throws PatternException {
           super(name);
  -        this.source = MapStackResolver.getResolver(source);
  +        this.source = source;
       }
   
       public void setParameters(Map parameterMap) {
  @@ -117,8 +117,8 @@
           Redirector     redirector     = PipelinesNode.getRedirector(env);
           SourceResolver resolver       = getSourceResolver(objectModel);
           List           mapStack       = context.getMapStack();
  -        String         resolvedSource = source.resolve(mapStack);
  -        Parameters     resolvedParams = MapStackResolver.buildParameters(this.parameters, mapStack);
  +        String         resolvedSource = source.resolve(mapStack, objectModel);
  +        Parameters     resolvedParams = VariableResolver.buildParameters(this.parameters, mapStack, objectModel);
   
           Map actionResult;
   
  
  
  
  1.4       +10 -8     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActionSetNode.java
  
  Index: ActionSetNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActionSetNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- ActionSetNode.java	22 Apr 2002 15:43:36 -0000	1.3
  +++ ActionSetNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -65,7 +65,7 @@
   import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.components.treeprocessor.NamedProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
   
  @@ -90,7 +90,7 @@
       private Action[] threadSafeActions;
   
       /** The src for each action */
  -    private MapStackResolver[] sources;
  +    private VariableResolver[] sources;
   
       /** The parameters for each action */
       private Map[] parameters;
  @@ -100,7 +100,7 @@
   
       public ActionSetNode(
         String name, String[] types, String[] actionNames,
  -      MapStackResolver[] sources, Map[] parameters) {
  +      VariableResolver[] sources, Map[] parameters) {
           super(name);
           this.types = types;
           this.actionNames = actionNames;
  @@ -152,10 +152,10 @@
               Action action;
   
               String actionName = actionNames[i];
  -            String source = sources[i].resolve(mapStack);
  +            String source = sources[i].resolve(mapStack, objectModel);
               if (actionName == null || actionName.equals(cocoonAction)) {
   
  -                Parameters actionParams = MapStackResolver.buildParameters(parameters[i], mapStack);
  +                Parameters actionParams = VariableResolver.buildParameters(parameters[i], mapStack, objectModel);
                   if (actionParams == Parameters.EMPTY_PARAMETERS) {
                       actionParams = params;
                   } else {
  @@ -194,11 +194,13 @@
       }
   
       public void dispose() {
  -        // Dispose all ThreadSafe actions
  +
  +        // Dispose ThreadSafe actions
           for (int i = 0; i < this.threadSafeActions.length; i++) {
               this.selector.release(this.threadSafeActions[i]);
           }
  -        this.manager.release(this.selector);
  +        
  +        this.manager.release(this.selector);        
       }
   
       /**
  
  
  
  1.3       +7 -5      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActionSetNodeBuilder.java
  
  Index: ActionSetNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ActionSetNodeBuilder.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- ActionSetNodeBuilder.java	27 Mar 2002 15:10:35 -0000	1.2
  +++ ActionSetNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.3
  @@ -57,7 +57,8 @@
   import org.apache.cocoon.acting.Action;
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   
   import java.util.*;
  @@ -93,7 +94,8 @@
   
                   actionTypes.add(type);
                   actionNames.add(childConfig.getAttribute("action", null));
  -                actionSources.add(MapStackResolver.getResolver(childConfig.getAttribute("src", null)));
  +                actionSources.add(VariableResolverFactory.getResolver(
  +                    childConfig.getAttribute("src", null), this.manager));
                   actionParameters.add(this.getParameters(childConfig));
   
               } else {
  @@ -107,8 +109,8 @@
           String[] types   = (String[])actionTypes.toArray(new String[actionTypes.size()]);
           String[] actions = (String[])actionNames.toArray(new String[actionNames.size()]);
           Map[]    parameters = (Map[])actionParameters.toArray(new Map[actionParameters.size()]);
  -        MapStackResolver[] sources =
  -            (MapStackResolver[])actionSources.toArray(new MapStackResolver[actionSources.size()]);
  +        VariableResolver[] sources =
  +            (VariableResolver[])actionSources.toArray(new VariableResolver[actionSources.size()]);
   
           ActionSetNode node = new ActionSetNode(actionSetName, types, actions, sources, parameters);
           this.treeBuilder.setupNode(node, config);
  
  
  
  1.5       +35 -28    xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/AggregateNode.java
  
  Index: AggregateNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/AggregateNode.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- AggregateNode.java	27 May 2002 13:14:52 -0000	1.4
  +++ AggregateNode.java	24 Jun 2002 20:32:19 -0000	1.5
  @@ -60,8 +60,8 @@
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.*;
   
  @@ -87,9 +87,9 @@
   
   public class AggregateNode extends AbstractProcessingNode {
   
  -    private MapStackResolver element;
  -    private MapStackResolver nsURI;
  -    private MapStackResolver nsPrefix;
  +    private VariableResolver element;
  +    private VariableResolver nsURI;
  +    private VariableResolver nsPrefix;
   
       /** All parts */
       private Part[] allParts;
  @@ -100,10 +100,10 @@
       /** View nodes to jump to */
       private Map viewNodes;
   
  -    public AggregateNode(String element, String nsURI, String nsPrefix) throws PatternException {
  -        this.element = MapStackResolver.getResolver(element);
  -        this.nsURI = MapStackResolver.getResolver(nsURI);
  -        this.nsPrefix = MapStackResolver.getResolver(nsPrefix);
  +    public AggregateNode(VariableResolver element, VariableResolver nsURI, VariableResolver nsPrefix) throws PatternException {
  +        this.element = element;
  +        this.nsURI = nsURI;
  +        this.nsPrefix = nsPrefix;
       }
   
       public void setParts(Part[] allParts, Map viewParts) {
  @@ -121,6 +121,7 @@
           boolean infoEnabled = getLogger().isInfoEnabled();
   
           List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
   
           // Setup aggregator
           ProcessingPipeline processingPipeline = context.getProcessingPipeline();
  @@ -129,9 +130,9 @@
   
           ContentAggregator aggregator = (ContentAggregator)processingPipeline.getGenerator();
           aggregator.setRootElement(
  -            this.element.resolve(mapStack),
  -            this.nsURI.resolve(mapStack),
  -            this.nsPrefix.resolve(mapStack)
  +            this.element.resolve(mapStack, objectModel),
  +            this.nsURI.resolve(mapStack, objectModel),
  +            this.nsPrefix.resolve(mapStack, objectModel)
           );
   
           // Get actual parts, potentially filtered by the view
  @@ -157,11 +158,11 @@
               Part part = actualParts[i];
               if (part != null) {
                   aggregator.addPart(
  -                    part.source.resolve(mapStack),
  -                    part.element.resolve(mapStack),
  -                    part.nsURI.resolve(mapStack),
  -                    part.stripRoot.resolve(mapStack),
  -                    part.nsPrefix.resolve(mapStack)
  +                    part.source.resolve(mapStack, objectModel),
  +                    part.element.resolve(mapStack, objectModel),
  +                    part.nsURI.resolve(mapStack, objectModel),
  +                    part.stripRoot.resolve(mapStack, objectModel),
  +                    part.nsPrefix.resolve(mapStack, objectModel)
                   );
               }
           }
  @@ -193,19 +194,25 @@
       }
   
       public static class Part {
  -        public Part(String source, String element, String nsURI, String nsPrefix, String stripRoot)
  +        public Part(
  +            VariableResolver source,
  +            VariableResolver element,
  +            VariableResolver nsURI,
  +            VariableResolver nsPrefix,
  +            VariableResolver stripRoot)
             throws PatternException {
  -            this.source = MapStackResolver.getResolver(source);
  -            this.element = MapStackResolver.getResolver(element);
  -            this.nsURI = MapStackResolver.getResolver(nsURI);
  -            this.nsPrefix = MapStackResolver.getResolver(nsPrefix);
  -            this.stripRoot = MapStackResolver.getResolver(stripRoot);
  +            this.source = source;
  +            this.element = element;
  +            this.nsURI = nsURI;
  +            this.nsPrefix = nsPrefix;
  +            this.stripRoot = stripRoot;
           }
   
  -        private MapStackResolver source;
  -        private MapStackResolver element;
  -        private MapStackResolver nsURI;
  -        private MapStackResolver nsPrefix;
  -        private MapStackResolver stripRoot;
  +        private VariableResolver source;
  +        private VariableResolver element;
  +        private VariableResolver nsURI;
  +        private VariableResolver nsPrefix;
  +        private VariableResolver stripRoot;
  +        
       }
   }
  
  
  
  1.4       +11 -10    xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/AggregateNodeBuilder.java
  
  Index: AggregateNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/AggregateNodeBuilder.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- AggregateNodeBuilder.java	22 Mar 2002 22:26:13 -0000	1.3
  +++ AggregateNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -54,8 +54,9 @@
   import org.apache.avalon.framework.configuration.ConfigurationException;
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
   
  @@ -80,9 +81,9 @@
   
           // Get root node data
           this.node = new AggregateNode(
  -            config.getAttribute("element"),
  -            config.getAttribute("ns", ""),
  -            config.getAttribute("prefix", "")
  +            VariableResolverFactory.getResolver(config.getAttribute("element"), this.manager),
  +            VariableResolverFactory.getResolver(config.getAttribute("ns", ""), this.manager),
  +            VariableResolverFactory.getResolver(config.getAttribute("prefix", ""), this.manager)
           );
           this.treeBuilder.setupNode(this.node, config);
   
  @@ -116,11 +117,11 @@
               checkNamespace(childConfig);
   
               AggregateNode.Part currentPart = new AggregateNode.Part(
  -                childConfig.getAttribute("src"),
  -                childConfig.getAttribute("element", ""),
  -                childConfig.getAttribute("ns", ""),
  -                childConfig.getAttribute("prefix", ""),
  -                childConfig.getAttribute("strip-root", "false")
  +                VariableResolverFactory.getResolver(childConfig.getAttribute("src"), this.manager),
  +                VariableResolverFactory.getResolver(childConfig.getAttribute("element", ""), this.manager),
  +                VariableResolverFactory.getResolver(childConfig.getAttribute("ns", ""), this.manager),
  +                VariableResolverFactory.getResolver(childConfig.getAttribute("prefix", ""), this.manager),
  +                VariableResolverFactory.getResolver(childConfig.getAttribute("strip-root", "false"), this.manager)
               );
               
               allParts.add(currentPart);
  
  
  
  1.2       +9 -8      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/CallFunctionNode.java
  
  Index: CallFunctionNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/CallFunctionNode.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- CallFunctionNode.java	19 May 2002 19:19:39 -0000	1.1
  +++ CallFunctionNode.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -16,10 +16,11 @@
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.sitemap.PipelinesNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   import org.apache.cocoon.environment.Environment;
   import org.apache.cocoon.environment.Redirector;
   import org.apache.cocoon.sitemap.PatternException;
  @@ -29,12 +30,12 @@
   {
     protected String functionName;
     protected List parameters;
  -  protected MapStackResolver resourceResolver;
  +  protected VariableResolver resourceResolver;
     protected ComponentManager manager;
     protected CategoryNode resources;
     protected String language;
   
  -  public static List resolveList(List expressions, List mapStack)
  +  public static List resolveList(List expressions, ComponentManager manager, List mapStack, Map objectModel)
       throws PatternException 
     {
       int size;
  @@ -45,7 +46,7 @@
   
       for (int i = 0; i < size; i++) {
         Interpreter.Argument arg = (Interpreter.Argument)expressions.get(i);
  -      String value = MapStackResolver.getResolver(arg.value).resolve(mapStack);
  +      String value = VariableResolverFactory.getResolver(arg.value, manager).resolve(mapStack, objectModel);
         result.add (new Interpreter.Argument(arg.name, value));
       }
   
  @@ -86,9 +87,9 @@
       }
   
       try {
  -      if (MapStackResolver.needsResolve(functionName)) {
  +      if (VariableResolverFactory.needsResolve(functionName)) {
           // Will always be resolved at invoke time
  -        this.resourceResolver = MapStackResolver.getResolver(functionName);
  +        this.resourceResolver = VariableResolverFactory.getResolver(functionName, this.manager);
         }
       }
       catch (PatternException ex) {
  @@ -117,12 +118,12 @@
   
       // Resolve parameters
       if (this.parameters != null)
  -      params = resolveList(this.parameters, context.getMapStack());
  +      params = resolveList(this.parameters, this.manager, context.getMapStack(), env.getObjectModel());
   
       String name = functionName;
       if (resourceResolver != null) {
         // Need to resolve the function name at runtime
  -      name = resourceResolver.resolve(context.getMapStack());
  +      name = resourceResolver.resolve(context.getMapStack(), env.getObjectModel());
       }
   
       InterpreterSelector selector
  
  
  
  1.4       +21 -9     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/CallNode.java
  
  Index: CallNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/CallNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- CallNode.java	31 Mar 2002 21:07:48 -0000	1.3
  +++ CallNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -51,6 +51,9 @@
   package org.apache.cocoon.components.treeprocessor.sitemap;
   
   import org.apache.avalon.framework.activity.Initializable;
  +import org.apache.avalon.framework.component.Composable;
  +import org.apache.avalon.framework.component.ComponentManager;
  +import org.apache.avalon.framework.component.ComponentException;
   
   import org.apache.cocoon.environment.Environment;
   
  @@ -59,9 +62,10 @@
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.List;
   import java.util.Map;
  @@ -73,7 +77,9 @@
    */
   
   public class CallNode extends AbstractProcessingNode
  -    implements Initializable, ParameterizableProcessingNode {
  +    implements Initializable, ParameterizableProcessingNode, Composable {
  +
  +    private ComponentManager manager;
   
       /** The parameters of this node */
       private Map parameters;
  @@ -81,13 +87,17 @@
       /** The 'resource' attribute */
       private String resourceName;
   
  -    private MapStackResolver resourceResolver;
  +    private VariableResolver resourceResolver;
   
       /** The category node */
       private CategoryNode resources;
   
       private ProcessingNode resourceNode;
   
  +    public void compose(ComponentManager manager) throws ComponentException {
  +        this.manager = manager;
  +    }
  +
       public void setParameters(Map parameterMap) {
           this.parameters = parameterMap;
       }
  @@ -98,21 +108,23 @@
       }
   
       public void initialize() throws Exception {
  -        if (MapStackResolver.needsResolve(this.resourceName)) {
  +        if (VariableResolverFactory.needsResolve(this.resourceName)) {
               // Will always be resolved at invoke time
  -            this.resourceResolver = MapStackResolver.getResolver(this.resourceName);
  +            this.resourceResolver = VariableResolverFactory.getResolver(this.resourceName, this.manager);
           } else {
               // Static name : get it now
  -            this.resourceNode = this.resources.getNodeByName(MapStackResolver.unescape(this.resourceName));
  +            this.resourceNode = this.resources.getNodeByName(VariableResolverFactory.unescape(this.resourceName));
           }
       }
   
       public final boolean invoke(Environment env, InvokeContext context)
         throws Exception {
   
  +        List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
           // Resolve parameters, but push them only once the resource name has been
           // resolved, otherwise it adds an unwanted nesting level
  -        Map params = MapStackResolver.resolveMap(this.parameters, context.getMapStack());
  +        Map params = VariableResolver.buildMap(this.parameters, mapStack, objectModel);
   
           if (this.resourceNode != null) {
               // Static resource name
  @@ -126,7 +138,7 @@
       
           } else {
               // Resolved resource name
  -            String name = this.resourceResolver.resolve(context.getMapStack());
  +            String name = this.resourceResolver.resolve(mapStack, objectModel);
               if (getLogger().isInfoEnabled()) {
                   getLogger().info("Calling resource " + name);
               }
  
  
  
  1.2       +9 -7      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ContinueNode.java
  
  Index: ContinueNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ContinueNode.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ContinueNode.java	19 May 2002 19:19:39 -0000	1.1
  +++ ContinueNode.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -13,11 +13,13 @@
   import org.apache.cocoon.components.flow.InterpreterSelector;
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.sitemap.PipelinesNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   import org.apache.cocoon.environment.Environment;
   import org.apache.cocoon.environment.Redirector;
   import org.apache.cocoon.sitemap.PatternException;
  +import org.apache.avalon.framework.activity.Disposable;
   
   public class ContinueNode
     extends AbstractProcessingNode
  @@ -25,7 +27,7 @@
   {
     protected String continuationId;
     protected List parameters;
  -  protected MapStackResolver continuationIdResolver;
  +  protected VariableResolver continuationIdResolver;
     protected ComponentManager manager;
   
     public ContinueNode(String contId)
  @@ -49,9 +51,9 @@
       try {
         // The continuation id should would need to be resolved at all
         // times, but who knows...
  -      if (MapStackResolver.needsResolve(continuationId)) {
  +      if (VariableResolverFactory.needsResolve(continuationId)) {
           this.continuationIdResolver
  -          = MapStackResolver.getResolver(continuationId);
  +          = VariableResolverFactory.getResolver(continuationId, this.manager);
         }
       }
       catch (PatternException ex) {
  @@ -71,13 +73,13 @@
   
       // Resolve parameters
       if (this.parameters != null)
  -      params = CallFunctionNode.resolveList(this.parameters,
  -                                            context.getMapStack());
  +      params = CallFunctionNode.resolveList(this.parameters, this.manager,
  +                                            context.getMapStack(), env.getObjectModel());
   
       String contId = continuationId;
   
       if (continuationIdResolver != null) {
  -      contId = continuationIdResolver.resolve(context.getMapStack());
  +      contId = continuationIdResolver.resolve(context.getMapStack(), env.getObjectModel());
       }
   
       InterpreterSelector selector
  
  
  
  1.2       +0 -11     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ContinueNodeBuilder.java
  
  Index: ContinueNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ContinueNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ContinueNodeBuilder.java	19 May 2002 19:19:39 -0000	1.1
  +++ ContinueNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -1,9 +1,6 @@
   package org.apache.cocoon.components.treeprocessor.sitemap;
   
   import org.apache.avalon.excalibur.component.ExcaliburComponentSelector;
  -import org.apache.avalon.framework.component.ComponentManager;
  -import org.apache.avalon.framework.component.ComponentSelector;
  -import org.apache.avalon.framework.component.Composable;
   import org.apache.avalon.framework.configuration.Configurable;
   import org.apache.avalon.framework.configuration.Configuration;
   import org.apache.avalon.framework.configuration.ConfigurationException;
  @@ -13,19 +10,11 @@
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   
   public class ContinueNodeBuilder extends AbstractProcessingNodeBuilder
  -  implements Composable
   {
     protected ContinueNode node;
  -  protected ComponentManager manager;
  -
  -  public void compose(ComponentManager manager)
  -  {
  -    this.manager = manager;
  -  }
   
     public ProcessingNode buildNode(Configuration config)
       throws Exception
  
  
  
  1.4       +9 -8      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/GenerateNode.java
  
  Index: GenerateNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/GenerateNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- GenerateNode.java	27 May 2002 13:14:52 -0000	1.3
  +++ GenerateNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -56,9 +56,9 @@
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.*;
   
  @@ -72,7 +72,7 @@
   
       private String generatorName;
   
  -    private MapStackResolver source;
  +    private VariableResolver source;
   
       private Map parameters;
   
  @@ -81,9 +81,9 @@
       /** The category node */
       private CategoryNode viewsNode;
   
  -    public GenerateNode(String name, String source) throws PatternException {
  +    public GenerateNode(String name, VariableResolver source) throws PatternException {
           this.generatorName = name;
  -        this.source = MapStackResolver.getResolver(source);
  +        this.source = source;
       }
   
       public void setParameters(Map parameterMap) {
  @@ -98,11 +98,12 @@
         throws Exception {
   
           List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
   
           context.getProcessingPipeline().setGenerator(
               this.generatorName,
  -            source.resolve(mapStack),
  -            MapStackResolver.buildParameters(this.parameters, mapStack)
  +            source.resolve(mapStack, objectModel),
  +            VariableResolver.buildParameters(this.parameters, mapStack, objectModel)
           );
   
           // Check view
  @@ -123,7 +124,7 @@
               }
           }
   
  -        // Return false to contine sitemap invocation
  +        // Return false to continue sitemap invocation
           return false;
       }
   }
  
  
  
  1.2       +7 -2      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/GenerateNodeBuilder.java
  
  Index: GenerateNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/GenerateNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- GenerateNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ GenerateNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -57,6 +57,8 @@
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
   
  @@ -79,7 +81,10 @@
   
           this.views = ((SitemapLanguage)this.treeBuilder).getViewsForStatement(Generator.ROLE, type, config);
   
  -        this.node = new GenerateNode(type, config.getAttribute("src", null));
  +        this.node = new GenerateNode(
  +            type,
  +            VariableResolverFactory.getResolver(config.getAttribute("src", null), this.manager)
  +        );
           return this.treeBuilder.setupNode(this.node, config);
       }
   
  
  
  
  1.4       +8 -8      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MatchNode.java
  
  Index: MatchNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MatchNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- MatchNode.java	29 Apr 2002 14:46:16 -0000	1.3
  +++ MatchNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -53,7 +53,7 @@
   import org.apache.cocoon.environment.Environment;
   import org.apache.cocoon.sitemap.PatternException;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.matching.Matcher;
   import org.apache.avalon.framework.activity.Disposable;
   import org.apache.avalon.framework.component.ComponentSelector;
  @@ -77,7 +77,7 @@
   implements ParameterizableProcessingNode, Composable, Disposable {
   
       /** The 'pattern' attribute */
  -    private MapStackResolver pattern;
  +    private VariableResolver pattern;
   
       /** The matcher, if it's ThreadSafe */
       private Matcher threadSafeMatcher;
  @@ -86,9 +86,9 @@
   
       private ComponentManager manager;
   
  -    public MatchNode(String name, String pattern) throws PatternException {
  +    public MatchNode(String name, VariableResolver pattern) throws PatternException {
           super(name);
  -        this.pattern = MapStackResolver.getResolver(pattern);
  +        this.pattern = pattern;
       }
   
       public void setParameters(Map parameterMap) {
  @@ -107,10 +107,10 @@
         throws Exception {
   
           List mapStack = context.getMapStack();
  -
  -        String resolvedPattern = pattern.resolve(mapStack);
           Map objectModel = env.getObjectModel();
  -        Parameters resolvedParams = MapStackResolver.buildParameters(this.parameters, mapStack);
  +
  +        String resolvedPattern = pattern.resolve(mapStack, objectModel);
  +        Parameters resolvedParams = VariableResolver.buildParameters(this.parameters, mapStack, objectModel);
   
           Map result = null;
   
  
  
  
  1.5       +7 -5      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MatchNodeBuilder.java
  
  Index: MatchNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MatchNodeBuilder.java,v
  retrieving revision 1.4
  retrieving revision 1.5
  diff -u -r1.4 -r1.5
  --- MatchNodeBuilder.java	22 Apr 2002 15:43:36 -0000	1.4
  +++ MatchNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.5
  @@ -60,9 +60,10 @@
   import org.apache.cocoon.matching.Matcher;
   import org.apache.cocoon.matching.PreparableMatcher;
   import org.apache.cocoon.components.treeprocessor.AbstractParentProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
   
  @@ -109,15 +110,16 @@
           // PreparableMatcher are only prepared if pattern doesn't need request-time resolution.
           boolean preparable =
               PreparableMatcher.class.isAssignableFrom(clazz) &&
  -            !MapStackResolver.needsResolve(pattern);
  +            !VariableResolverFactory.needsResolve(pattern);
   
           // Instanciate appropriate node
           SimpleSelectorProcessingNode node;
  +        VariableResolver patternResolver = VariableResolverFactory.getResolver(pattern, this.manager);
   
           if (preparable) {
  -            node = new PreparableMatchNode(type, pattern);
  +            node = new PreparableMatchNode(type, VariableResolverFactory.unescape(pattern));
           } else {
  -            node = new MatchNode(type, pattern);
  +            node = new MatchNode(type, patternResolver);
           }
   
           this.treeBuilder.setupNode(node, config);
  
  
  
  1.7       +9 -10     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MountNode.java
  
  Index: MountNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MountNode.java,v
  retrieving revision 1.6
  retrieving revision 1.7
  diff -u -r1.6 -r1.7
  --- MountNode.java	13 Jun 2002 09:28:15 -0000	1.6
  +++ MountNode.java	24 Jun 2002 20:32:19 -0000	1.7
  @@ -54,10 +54,10 @@
   import org.apache.avalon.framework.component.ComponentManager;
   import org.apache.avalon.framework.component.Composable;
   import org.apache.cocoon.components.pipeline.ProcessingPipeline;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.TreeProcessor;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   import org.apache.cocoon.environment.Environment;
   import org.apache.cocoon.sitemap.PatternException;
   import java.util.*;
  @@ -71,10 +71,10 @@
   public class MountNode extends AbstractProcessingNode implements Composable {
   
       /** The 'uri-prefix' attribute */
  -    private MapStackResolver prefix;
  +    private VariableResolver prefix;
   
       /** The 'src' attribute */
  -    private MapStackResolver source;
  +    private VariableResolver source;
   
       /** Processors for sources */
       private Map processors = new HashMap();
  @@ -88,10 +88,8 @@
       /** The component manager to be used by the mounted processor */
       private ComponentManager manager;
   
  -    public MountNode(String prefix, String source, String language, TreeProcessor parentProcessor)
  -      throws PatternException {
  -        this.prefix = MapStackResolver.getResolver(prefix);
  -        this.source = MapStackResolver.getResolver(source);
  +    public MountNode(VariableResolver prefix, VariableResolver source, String language, TreeProcessor parentProcessor) {        this.prefix = prefix;
  +        this.source = source;
           this.language = language;
           this.parentProcessor = parentProcessor;
       }
  @@ -104,14 +102,15 @@
         throws Exception {
   
           List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
   
  -        String resolvedSource = this.source.resolve(mapStack);
  +        String resolvedSource = this.source.resolve(mapStack, objectModel);
           TreeProcessor processor = (TreeProcessor)processors.get(resolvedSource);
           if (processor == null) {
               processor = getProcessor(env, resolvedSource);
           }
   
  -        String resolvedPrefix = this.prefix.resolve(mapStack);
  +        String resolvedPrefix = this.prefix.resolve(mapStack, objectModel);
   
           String oldPrefix = env.getURIPrefix();
           String oldURI    = env.getURI();
  
  
  
  1.2       +5 -3      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MountNodeBuilder.java
  
  Index: MountNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/MountNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- MountNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ MountNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -55,6 +55,8 @@
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   /**
    *
  @@ -72,8 +74,8 @@
       public ProcessingNode buildNode(Configuration config) throws Exception {
   
           MountNode node = new MountNode(
  -            config.getAttribute("uri-prefix"),
  -            config.getAttribute("src"),
  +            VariableResolverFactory.getResolver(config.getAttribute("uri-prefix"), manager),
  +            VariableResolverFactory.getResolver(config.getAttribute("src"), manager),
               config.getAttribute("language", null),
               this.treeBuilder.getProcessor()
           );
  
  
  
  1.11      +2 -3      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java
  
  Index: PipelineNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PipelineNode.java,v
  retrieving revision 1.10
  retrieving revision 1.11
  diff -u -r1.10 -r1.11
  --- PipelineNode.java	17 Jun 2002 01:21:19 -0000	1.10
  +++ PipelineNode.java	24 Jun 2002 20:32:19 -0000	1.11
  @@ -64,7 +64,6 @@
   import org.apache.cocoon.components.notification.Notifying;
   import org.apache.cocoon.components.notification.NotifyingBuilder;
   import org.apache.cocoon.components.treeprocessor.AbstractParentProcessingNode;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  @@ -152,7 +151,7 @@
               return false;
           }
   
  -        context.inform(this.processingPipeline, this.parameters);
  +        context.inform(this.processingPipeline, this.parameters, env.getObjectModel());
   
           try {
               if (invokeNodes(children, env, context)) {
  
  
  
  1.4       +7 -4      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PreparableMatchNode.java
  
  Index: PreparableMatchNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/PreparableMatchNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- PreparableMatchNode.java	29 Apr 2002 14:46:16 -0000	1.3
  +++ PreparableMatchNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -53,7 +53,8 @@
   import org.apache.cocoon.environment.Environment;
   import org.apache.cocoon.sitemap.PatternException;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   import org.apache.cocoon.matching.Matcher;
   import org.apache.cocoon.matching.PreparableMatcher;
   import org.apache.avalon.framework.activity.Disposable;
  @@ -110,7 +111,7 @@
           }
   
           try {
  -            this.preparedPattern = matcher.preparePattern(MapStackResolver.unescape(this.pattern));
  +            this.preparedPattern = matcher.preparePattern(this.pattern);
   
           } catch(PatternException pe) {
               String msg = "Invalid pattern '" + this.pattern + "' for matcher at " + this.getLocation();
  @@ -128,7 +129,9 @@
         throws Exception {
   
           Map objectModel = env.getObjectModel();
  -        Parameters resolvedParams = MapStackResolver.buildParameters(this.parameters, context.getMapStack());
  +        Parameters resolvedParams = VariableResolver.buildParameters(
  +            this.parameters, context.getMapStack(), objectModel
  +        );
   
           Map result = null;
   
  
  
  
  1.3       +8 -8      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ReadNode.java
  
  Index: ReadNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ReadNode.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- ReadNode.java	27 May 2002 13:14:52 -0000	1.2
  +++ ReadNode.java	24 Jun 2002 20:32:19 -0000	1.3
  @@ -52,12 +52,11 @@
   
   import org.apache.cocoon.components.pipeline.ProcessingPipeline;
   import org.apache.cocoon.environment.Environment;
  -import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.List;
   import java.util.Map;
  @@ -72,7 +71,7 @@
   
       private String readerName;
   
  -    private MapStackResolver source;
  +    private VariableResolver source;
   
       private String mimeType;
   
  @@ -87,9 +86,9 @@
        * @param mimeType the mime-type, or <code>null</code> not specified.
        * @param statusCode the HTTP response status code, or <code>-1</code> if not specified.
        */
  -    public ReadNode(String name, String source, String mimeType, int statusCode) throws PatternException {
  +    public ReadNode(String name, VariableResolver source, String mimeType, int statusCode) {
           this.readerName = name;
  -        this.source = MapStackResolver.getResolver(source);
  +        this.source = source;
           this.mimeType = mimeType;
           this.statusCode = statusCode;
       }
  @@ -102,13 +101,14 @@
         throws Exception {
   
           List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
   
           ProcessingPipeline pipeline = context.getProcessingPipeline();
   
           pipeline.setReader(
               this.readerName,
  -            source.resolve(mapStack),
  -            MapStackResolver.buildParameters(this.parameters, mapStack),
  +            source.resolve(mapStack, objectModel),
  +            VariableResolver.buildParameters(this.parameters, mapStack, objectModel),
               this.mimeType
           );
   
  
  
  
  1.2       +4 -2      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ReadNodeBuilder.java
  
  Index: ReadNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ReadNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ReadNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ ReadNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -58,6 +58,8 @@
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   /**
    *
  @@ -73,7 +75,7 @@
   
           ReadNode node = new ReadNode(
               type,
  -            config.getAttribute("src", null),
  +            VariableResolverFactory.getResolver(config.getAttribute("src", null), this.manager),
               config.getAttribute("mime-type", null),
               config.getAttributeAsInteger("status-code", -1)
           );
  
  
  
  1.2       +14 -5     xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/RedirectToNodeBuilder.java
  
  Index: RedirectToNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/RedirectToNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- RedirectToNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ RedirectToNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -57,10 +57,14 @@
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
  +import org.apache.avalon.framework.component.Composable;
  +import org.apache.avalon.framework.component.ComponentManager;
  +import org.apache.avalon.framework.component.ComponentException;
   
   /**
    *
  @@ -69,11 +73,16 @@
    */
   
   public class RedirectToNodeBuilder extends AbstractProcessingNodeBuilder
  -  implements LinkedProcessingNodeBuilder {
  +  implements LinkedProcessingNodeBuilder, Composable {
   
  +    private ComponentManager manager;
       private CallNode callNode;
       private String resourceName;
   
  +    public void compose(ComponentManager manager) throws ComponentException {
  +        this.manager = manager;
  +    }
  +
       /** This builder has no parameters -- return <code>false</code> */
       protected boolean hasParameters() {
           return false;
  @@ -94,14 +103,14 @@
               String target = config.getAttribute("target", null);
               if (target != null) {
                   Map params = new HashMap(1);
  -                params.put("target", MapStackResolver.getResolver(target));
  +                params.put("target", VariableResolverFactory.getResolver(target, this.manager));
                   this.callNode.setParameters(params);
               }
               return this.callNode;
               
           } else {
               ProcessingNode URINode = new RedirectToURINode(
  -                config.getAttribute("uri"),
  +                VariableResolverFactory.getResolver(config.getAttribute("uri"), this.manager),
                   config.getAttributeAsBoolean("session", false)
               );
               return this.treeBuilder.setupNode(URINode, config);
  
  
  
  1.3       +6 -6      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/RedirectToURINode.java
  
  Index: RedirectToURINode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/RedirectToURINode.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- RedirectToURINode.java	27 Mar 2002 15:13:40 -0000	1.2
  +++ RedirectToURINode.java	24 Jun 2002 20:32:19 -0000	1.3
  @@ -57,8 +57,8 @@
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.List;
   import java.util.Map;
  @@ -72,17 +72,17 @@
   public class RedirectToURINode extends AbstractProcessingNode {
   
       /** The 'uri' attribute */
  -    private MapStackResolver uri;
  +    private VariableResolver uri;
   
       private boolean keepSession;
   
  -    public RedirectToURINode(String uri, boolean keepSession) throws PatternException {
  -        this.uri = MapStackResolver.getResolver(uri);
  +    public RedirectToURINode(VariableResolver uri, boolean keepSession) throws PatternException {
  +        this.uri = uri;
       }
   
       public final boolean invoke(Environment env, InvokeContext context)
         throws Exception {
  -        String resolvedURI = uri.resolve(context.getMapStack());
  +        String resolvedURI = uri.resolve(context.getMapStack(), env.getObjectModel());
   
           if (getLogger().isInfoEnabled()) {
               getLogger().info("Redirecting to '" + resolvedURI + "' at " + this.getLocation());
  
  
  
  1.2       +0 -8      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ScriptNodeBuilder.java
  
  Index: ScriptNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/ScriptNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- ScriptNodeBuilder.java	19 May 2002 19:19:39 -0000	1.1
  +++ ScriptNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -12,19 +12,11 @@
   import org.apache.cocoon.components.treeprocessor.CategoryNode;
   import org.apache.cocoon.components.treeprocessor.CategoryNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   
   public class ScriptNodeBuilder extends AbstractProcessingNodeBuilder
  -  implements Composable
   {
     protected ScriptNode node;
  -  protected ComponentManager manager;
  -
  -  public void compose(ComponentManager manager)
  -  {
  -    this.manager = manager;
  -  }
   
     public ProcessingNode buildNode(Configuration config)
       throws Exception
  
  
  
  1.4       +7 -7      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SelectNode.java
  
  Index: SelectNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SelectNode.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- SelectNode.java	22 Apr 2002 15:43:36 -0000	1.3
  +++ SelectNode.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -66,10 +66,10 @@
   import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.List;
   import java.util.Map;
  @@ -91,7 +91,7 @@
   
       private ProcessingNode[][] whenNodes;
   
  -    private MapStackResolver[] whenTests;
  +    private VariableResolver[] whenTests;
   
       private ProcessingNode[] otherwhiseNodes;
   
  @@ -105,7 +105,7 @@
           this.parameters = parameterMap;
       }
   
  -    public void setCases(ProcessingNode[][] whenNodes, MapStackResolver[] whenTests, ProcessingNode[] otherwhiseNodes) {
  +    public void setCases(ProcessingNode[][] whenNodes, VariableResolver[] whenTests, ProcessingNode[] otherwhiseNodes) {
           this.whenNodes = whenNodes;
           this.whenTests = whenTests;
           this.otherwhiseNodes = otherwhiseNodes;
  @@ -125,14 +125,14 @@
           // Prepare data needed by the action
           Map objectModel = env.getObjectModel();
           List mapStack   = context.getMapStack();
  -        Parameters resolvedParams = MapStackResolver.buildParameters(this.parameters, mapStack);
  +        Parameters resolvedParams = VariableResolver.buildParameters(this.parameters, mapStack, objectModel);
   
           // If selector is ThreadSafe, avoid select() and try/catch block (faster !)
           if (this.threadSafeSelector != null) {
   
               for (int i = 0; i < this.whenTests.length; i++) {
                   if (this.threadSafeSelector.select(
  -                        whenTests[i].resolve(mapStack),
  +                        whenTests[i].resolve(mapStack, objectModel),
                           objectModel,
                           resolvedParams)) {
                       return invokeNodes(this.whenNodes[i], env, context);
  @@ -151,7 +151,7 @@
   
                   for (int i = 0; i < this.whenTests.length; i++) {
                       if (selector.select(
  -                            whenTests[i].resolve(mapStack),
  +                            whenTests[i].resolve(mapStack, objectModel),
                               objectModel,
                               resolvedParams)) {
                           return invokeNodes(this.whenNodes[i], env, context);
  
  
  
  1.4       +7 -4      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SelectNodeBuilder.java
  
  Index: SelectNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SelectNodeBuilder.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- SelectNodeBuilder.java	22 Apr 2002 15:43:36 -0000	1.3
  +++ SelectNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.4
  @@ -58,9 +58,10 @@
   import org.apache.cocoon.selection.SwitchSelector;
   
   import org.apache.cocoon.components.treeprocessor.AbstractParentProcessingNodeBuilder;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
   import org.apache.avalon.framework.component.Recomposable;
  @@ -107,7 +108,9 @@
               if ("when".equals(name)) {
   
                   checkNamespace(childConfig);
  -                whenTests.add(MapStackResolver.getResolver(childConfig.getAttribute("test")));
  +                whenTests.add(
  +                    VariableResolverFactory.getResolver(childConfig.getAttribute("test"), this.manager)
  +                );
                   whenChildren.add(buildChildNodes(childConfig));
   
               } else if ("otherwise".equals(name)) {
  @@ -133,7 +136,7 @@
           }
   
           ProcessingNode[][] whenChildrenNodes = (ProcessingNode[][])whenChildren.toArray(new ProcessingNode[0][0]);
  -        MapStackResolver[] whenResolvers = (MapStackResolver[])whenTests.toArray(new MapStackResolver[whenTests.size()]);
  +        VariableResolver[] whenResolvers = (VariableResolver[])whenTests.toArray(new VariableResolver[whenTests.size()]);
   
           // Get the type and class for this selector
           ComponentsSelector compSelector = (ComponentsSelector)this.manager.lookup(SELECTOR_ROLE);
  
  
  
  1.3       +7 -7      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SwitchSelectNode.java
  
  Index: SwitchSelectNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/SwitchSelectNode.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- SwitchSelectNode.java	22 Apr 2002 15:43:36 -0000	1.2
  +++ SwitchSelectNode.java	24 Jun 2002 20:32:19 -0000	1.3
  @@ -67,10 +67,10 @@
   import org.apache.cocoon.sitemap.PatternException;
   
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
   import org.apache.cocoon.components.treeprocessor.SimpleSelectorProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.List;
   import java.util.Map;
  @@ -92,7 +92,7 @@
   
       private ProcessingNode[][] whenNodes;
   
  -    private MapStackResolver[] whenTests;
  +    private VariableResolver[] whenTests;
   
       private ProcessingNode[] otherwhiseNodes;
   
  @@ -106,7 +106,7 @@
           this.parameters = parameterMap;
       }
   
  -    public void setCases(ProcessingNode[][] whenNodes, MapStackResolver[] whenTests, ProcessingNode[] otherwhiseNodes) {
  +    public void setCases(ProcessingNode[][] whenNodes, VariableResolver[] whenTests, ProcessingNode[] otherwhiseNodes) {
           this.whenNodes = whenNodes;
           this.whenTests = whenTests;
           this.otherwhiseNodes = otherwhiseNodes;
  @@ -127,7 +127,7 @@
           // Prepare data needed by the action
           Map objectModel = env.getObjectModel();
           List mapStack   = context.getMapStack();
  -        Parameters resolvedParams = MapStackResolver.buildParameters(this.parameters, mapStack);
  +        Parameters resolvedParams = VariableResolver.buildParameters(this.parameters, mapStack, objectModel);
   
           // If selector is ThreadSafe, avoid select() and try/catch block (faster !)
           if (this.threadSafeSelector != null) {
  @@ -135,7 +135,7 @@
               Object ctx = this.threadSafeSelector.getSelectorContext(objectModel, resolvedParams);
   
               for (int i = 0; i < this.whenTests.length; i++) {
  -                if (this.threadSafeSelector.select(whenTests[i].resolve(mapStack), ctx)) {
  +                if (this.threadSafeSelector.select(whenTests[i].resolve(mapStack, objectModel), ctx)) {
                       return invokeNodes(this.whenNodes[i], env, context);
                   }
               }
  @@ -154,7 +154,7 @@
               try {
   
                   for (int i = 0; i < this.whenTests.length; i++) {
  -                    if (selector.select(whenTests[i].resolve(mapStack), ctx)) {
  +                    if (selector.select(whenTests[i].resolve(mapStack, objectModel), ctx)) {
                           return invokeNodes(this.whenNodes[i], env, context);
                       }
                   }
  
  
  
  1.3       +8 -7      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/TransformNode.java
  
  Index: TransformNode.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/TransformNode.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- TransformNode.java	27 May 2002 13:14:52 -0000	1.2
  +++ TransformNode.java	24 Jun 2002 20:32:19 -0000	1.3
  @@ -55,9 +55,9 @@
   
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNode;
   import org.apache.cocoon.components.treeprocessor.InvokeContext;
  -import org.apache.cocoon.components.treeprocessor.MapStackResolver;
   import org.apache.cocoon.components.treeprocessor.ParameterizableProcessingNode;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
   
   import java.util.List;
   import java.util.Map;
  @@ -72,15 +72,15 @@
   
       private String transformerName;
   
  -    private MapStackResolver source;
  +    private VariableResolver source;
   
       private Map parameters;
   
       private Map views;
   
  -    public TransformNode(String name, String source) throws PatternException {
  +    public TransformNode(String name, VariableResolver source) throws PatternException {
           this.transformerName = name;
  -        this.source = MapStackResolver.getResolver(source);
  +        this.source = source;
       }
   
       public void setParameters(Map parameterMap) {
  @@ -95,11 +95,12 @@
         throws Exception {
   
           List mapStack = context.getMapStack();
  +        Map objectModel = env.getObjectModel();
   
           context.getProcessingPipeline().addTransformer(
               this.transformerName,
  -            source.resolve(mapStack),
  -            MapStackResolver.buildParameters(this.parameters, mapStack)
  +            source.resolve(mapStack, objectModel),
  +            VariableResolver.buildParameters(this.parameters, mapStack, objectModel)
           );
   
           // Check view
  
  
  
  1.2       +7 -2      xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/TransformNodeBuilder.java
  
  Index: TransformNodeBuilder.java
  ===================================================================
  RCS file: /home/cvs/xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/sitemap/TransformNodeBuilder.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- TransformNodeBuilder.java	5 Mar 2002 08:26:23 -0000	1.1
  +++ TransformNodeBuilder.java	24 Jun 2002 20:32:19 -0000	1.2
  @@ -57,6 +57,8 @@
   import org.apache.cocoon.components.treeprocessor.AbstractProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.LinkedProcessingNodeBuilder;
   import org.apache.cocoon.components.treeprocessor.ProcessingNode;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolver;
  +import org.apache.cocoon.components.treeprocessor.variables.VariableResolverFactory;
   
   import java.util.*;
   
  @@ -79,7 +81,10 @@
   
           this.views = ((SitemapLanguage)this.treeBuilder).getViewsForStatement(Transformer.ROLE, type, config);
   
  -        this.node = new TransformNode(type, config.getAttribute("src", null));
  +        this.node = new TransformNode(
  +            type,
  +            VariableResolverFactory.getResolver(config.getAttribute("src", null), this.manager)
  +        );
           return this.treeBuilder.setupNode(node, config);
       }
   
  
  
  
  1.1                  xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/variables/NOPVariableResolver.java
  
  Index: NOPVariableResolver.java
  ===================================================================
  /*
  
   ============================================================================
                     The Apache Software License, Version 1.1
   ============================================================================
  
   Copyright (C) 1999-2002 The Apache Software Foundation. All rights reserved.
  
   Redistribution and use in source and binary forms, with or without modifica-
   tion, are permitted provided that the following conditions are met:
  
   1. Redistributions of  source code must  retain the above copyright  notice,
      this list of conditions and the following disclaimer.
  
   2. Redistributions in binary form must reproduce the above copyright notice,
      this list of conditions and the following disclaimer in the documentation
      and/or other materials provided with the distribution.
  
   3. The end-user documentation included with the redistribution, if any, must
      include  the following  acknowledgment:  "This product includes  software
      developed  by the  Apache Software Foundation  (http://www.apache.org/)."
      Alternately, this  acknowledgment may  appear in the software itself,  if
      and wherever such third-party acknowledgments normally appear.
  
   4. The names "Apache Cocoon" and  "Apache Software Foundation" must  not  be
      used to  endorse or promote  products derived from  this software without
      prior written permission. For written permission, please contact
      apache@apache.org.
  
   5. Products  derived from this software may not  be called "Apache", nor may
      "Apache" appear  in their name,  without prior written permission  of the
      Apache Software Foundation.
  
   THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
   INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
   FITNESS  FOR A PARTICULAR  PURPOSE ARE  DISCLAIMED.  IN NO  EVENT SHALL  THE
   APACHE SOFTWARE  FOUNDATION  OR ITS CONTRIBUTORS  BE LIABLE FOR  ANY DIRECT,
   INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR CONSEQUENTIAL  DAMAGES (INCLU-
   DING, BUT NOT LIMITED TO, PROCUREMENT  OF SUBSTITUTE GOODS OR SERVICES; LOSS
   OF USE, DATA, OR  PROFITS; OR BUSINESS  INTERRUPTION)  HOWEVER CAUSED AND ON
   ANY  THEORY OF LIABILITY,  WHETHER  IN CONTRACT,  STRICT LIABILITY,  OR TORT
   (INCLUDING  NEGLIGENCE OR  OTHERWISE) ARISING IN  ANY WAY OUT OF THE  USE OF
   THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
  
   This software  consists of voluntary contributions made  by many individuals
   on  behalf of the Apache Software  Foundation and was  originally created by
   Stefano Mazzocchi  <st...@apache.org>. For more  information on the Apache
   Software Foundation, please see <http://www.apache.org/>.
  
  */
  package org.apache.cocoon.components.treeprocessor.variables;
  
  import java.util.List;
  import java.util.Map;
  
  /**
   * No-op implementation of {@link VariableResolver} for constant expressions
   */
  
  public class NOPVariableResolver extends VariableResolver {
  
      private String originalExpr = null;
      private String expression = null;
  
      public NOPVariableResolver(String expression) {
          if (expression != null) {
              // Keep the original unescaped expression
              this.originalExpr = expression;
              this.expression = VariableResolverFactory.unescape(expression);
          }
      }
  
      public String toString() {
          return this.originalExpr;
      }
  
      public final String resolve(List mapStack, Map objectModel) {
          return this.expression;
      }
      
      public final void release() {
          // Nothing to do
      }
  }
  
  
  1.1                  xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/variables/PreparedVariableResolver.java
  
  Index: PreparedVariableResolver.java
  ===================================================================
  /*
  
   ============================================================================
                     The Apache Software License, Version 1.1
   ============================================================================
  
   Copyright (C) 1999-2002 The Apache Software Foundation. All rights reserved.
  
   Redistribution and use in source and binary forms, with or without modifica-
   tion, are permitted provided that the following conditions are met:
  
   1. Redistributions of  source code must  retain the above copyright  notice,
      this list of conditions and the following disclaimer.
  
   2. Redistributions in binary form must reproduce the above copyright notice,
      this list of conditions and the following disclaimer in the documentation
      and/or other materials provided with the distribution.
  
   3. The end-user documentation included with the redistribution, if any, must
      include  the following  acknowledgment:  "This product includes  software
      developed  by the  Apache Software Foundation  (http://www.apache.org/)."
      Alternately, this  acknowledgment may  appear in the software itself,  if
      and wherever such third-party acknowledgments normally appear.
  
   4. The names "Apache Cocoon" and  "Apache Software Foundation" must  not  be
      used to  endorse or promote  products derived from  this software without
      prior written permission. For written permission, please contact
      apache@apache.org.
  
   5. Products  derived from this software may not  be called "Apache", nor may
      "Apache" appear  in their name,  without prior written permission  of the
      Apache Software Foundation.
  
   THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
   INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
   FITNESS  FOR A PARTICULAR  PURPOSE ARE  DISCLAIMED.  IN NO  EVENT SHALL  THE
   APACHE SOFTWARE  FOUNDATION  OR ITS CONTRIBUTORS  BE LIABLE FOR  ANY DIRECT,
   INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR CONSEQUENTIAL  DAMAGES (INCLU-
   DING, BUT NOT LIMITED TO, PROCUREMENT  OF SUBSTITUTE GOODS OR SERVICES; LOSS
   OF USE, DATA, OR  PROFITS; OR BUSINESS  INTERRUPTION)  HOWEVER CAUSED AND ON
   ANY  THEORY OF LIABILITY,  WHETHER  IN CONTRACT,  STRICT LIABILITY,  OR TORT
   (INCLUDING  NEGLIGENCE OR  OTHERWISE) ARISING IN  ANY WAY OUT OF THE  USE OF
   THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
  
   This software  consists of voluntary contributions made  by many individuals
   on  behalf of the Apache Software  Foundation and was  originally created by
   Stefano Mazzocchi  <st...@apache.org>. For more  information on the Apache
   Software Foundation, please see <http://www.apache.org/>.
  
  */
  package org.apache.cocoon.components.treeprocessor.variables;
  
  import org.apache.avalon.framework.activity.Disposable;
  import org.apache.avalon.framework.component.ComponentException;
  import org.apache.avalon.framework.component.ComponentManager;
  import org.apache.avalon.framework.component.ComponentSelector;
  import org.apache.avalon.framework.configuration.ConfigurationException;
  import org.apache.avalon.framework.thread.ThreadSafe;
  
  import org.apache.cocoon.components.modules.input.InputModule;
  import org.apache.cocoon.sitemap.PatternException;
  
  import java.util.ArrayList;
  import java.util.List;
  import java.util.Map;
  
  /**
   * Prepared implementation of {@link VariableResolver} for fast evaluation.
   */
  
  public class PreparedVariableResolver extends VariableResolver implements Disposable {
      
      private String originalExpr;
      private ComponentManager manager;
      private ComponentSelector selector;
      
      private List items = new ArrayList();
      
      // Special constants used for levels
      // later : final int ROOT_VAR = 0;
      static final int LITERAL = -1;
      static final int THREADSAFE_MODULE = -2;
      static final int STATEFUL_MODULE = -3;
      
      public PreparedVariableResolver(String expr, ComponentManager manager) throws PatternException {
          
          this.originalExpr = expr;
          this.manager = manager;
          
  
          int length = expr.length();
          int prev = 0; // position after last closing brace
  
          compile : while(prev < length) {
              // find next unescaped '{'
              int pos = prev;
              while(pos < length &&
                    (pos = expr.indexOf('{', pos)) != -1 &&
                    (pos != 0 && expr.charAt(pos - 1) == '\\')) {
                  pos++;
              }
  
              if (pos >= length || pos == -1) {
                  // no more braces : add ending literal
                  if (prev < length) {
                      addLiteral(expr.substring(prev));
                  }
                  break compile;
              }
  
              // Pass closing brace
              pos++;
  
              // Add litteral strings between closing and next opening brace
              if (prev < pos-1) {
                  addLiteral(expr.substring(prev, pos - 1));
              }
  
              int end = expr.indexOf('}', pos);
              if (end == -1) {
                  throw new PatternException("Unmatched '{' in " + expr);
              }
  
              int colon = expr.indexOf(':', pos);
              if (colon != -1 && colon < end) {
                  if (expr.startsWith("sitemap:", pos)) {
                      // Explicit prefix for sitemap variable
                      String variable = expr.substring(pos + "sitemap:".length(), end);
                      addSitemapVariable(variable);
                  } else {
                      
                      // Module used
                      String module = expr.substring(pos, colon);
                      String variable = expr.substring(colon + 1, end);
                      addModuleVariable(module, variable);
                  }
              } else {
                  // Unprefixed name : sitemap variable
                  addSitemapVariable(expr.substring(pos, end));
              }
  
              prev = end + 1;
          }
      }
      
      private void addLiteral(String litteral) {
          this.items.add(new Integer(LITERAL));
          this.items.add(litteral);
      }
      
      private void addSitemapVariable(String variable) {
          // Find level
          int level = 1; // Start at 1 since it will be substracted from list.size()
          int pos = 0;
          while(variable.startsWith("../", pos)) {
              level++;
              pos += "../".length();
          }
          this.items.add(new Integer(level));
          this.items.add(variable.substring(pos));
      }
      
      private void addModuleVariable(String moduleName, String variable) throws PatternException {
          if (this.selector == null) {
              try {
                  // First access to a module : lookup selector
                  this.selector = (ComponentSelector)this.manager.lookup(InputModule.ROLE + "Selector");
              } catch(ComponentException ce) {
                  throw new PatternException("Cannot access input modules selector", ce);
              }
          }
          
          // Get the module
          InputModule module;
          try {
              module = (InputModule)this.selector.select(moduleName);
          } catch(ComponentException ce) {
              throw new PatternException("Cannot get InputModule named '" + moduleName +
                  "' in expression '" + this.originalExpr + "'", ce);
          }
          
          // Is this module threadsafe ?
          if (module instanceof ThreadSafe) {
              this.items.add(new Integer(this.THREADSAFE_MODULE));
              this.items.add(module);
              this.items.add(variable);
          } else {
              // Statefull module : release it
              this.selector.release(module);
              this.items.add(new Integer(this.STATEFUL_MODULE));
              this.items.add(moduleName);
              this.items.add(variable);
          }
      }
      
      public final String resolve(List mapStack, Map objectModel) throws PatternException {
          StringBuffer result = new StringBuffer();
          
          int stackSize = mapStack.size();
          
          for (int i = 0; i < this.items.size(); i++) {
              int type = ((Integer)this.items.get(i)).intValue();
              
              if (type > 0) {
                  // relative sitemap variable
                  if (type > stackSize) {
                      throw new PatternException("Error while evaluating '" + this.originalExpr +
                          "' : not so many levels");
                  }
  
                  Object key = this.items.get(++i);
                  Object value = ((Map)mapStack.get(stackSize - type)).get(key);
                  if (value != null) {
                      result.append(value);
                  }
                  
              } else {
                  // other variable types
                  switch(type) {
                      case LITERAL :
                          result.append(items.get(++i));
                      break;
                      
                      case THREADSAFE_MODULE :
                      {
                          InputModule module = (InputModule)items.get(++i);
                          String variable = (String)items.get(++i);
                          
                          try {                    
                              Object value = module.getAttribute(variable, null, objectModel);
                              
                              if (value != null) {
                                  result.append(value);
                              }
      
                          } catch(ConfigurationException confEx) {
                              throw new PatternException("Cannot get variable '" + variable +
                                  "' in expression '" + this.originalExpr + "'", confEx);
                          }
                      }
                      break;
                      
                      case STATEFUL_MODULE :
                      {
                          InputModule module = null;
                          String moduleName = (String)items.get(++i);
                          String variableName = (String)items.get(++i);
                          try {
                              module = (InputModule)this.selector.select(moduleName);
                              
                              Object value = module.getAttribute(variableName, null, objectModel);
                              
                              if (value != null) {
                                  result.append(value);
                              }
                              
                          } catch(ComponentException compEx) {
                              throw new PatternException("Cannot get module '" + moduleName +
                                  "' in expression '" + this.originalExpr + "'", compEx);
                                  
                          } catch(ConfigurationException confEx) {
                              throw new PatternException("Cannot get variable '" + variableName +
                                  "' in expression '" + this.originalExpr + "'", confEx);
                                  
                          } finally {
                              this.selector.release(module);
                          }
                      }
                      break;
                  }
              }
          }
          
          return result.toString();
          
      }
      
      public final void dispose() {
          if (this.selector == null) {
              // No module in this expression, or already released
              return;
          }
          
          for (int i = 0; i < this.items.size(); i++) {
              int type = ((Integer)this.items.get(i)).intValue();
              
              switch(type) {
                  case LITERAL :
                      i++; // literal string
                  break;
                  
                  case THREADSAFE_MODULE :
                      i++; // module
                      this.selector.release((InputModule)this.items.get(i));
                      i++; // variable
                  break;
                  
                  case STATEFUL_MODULE :
                      i += 2; // module name, variable
                  break;
              }
          }
          
          this.manager.release(this.selector);
          this.selector = null;
          this.manager = null;
      }
      
  //        public void dump() {
  //            System.out.println(this.originalExpr + " compiled in :");
  //            for (int i = 0; i < this.strings.length; i++) {
  //                System.out.print("[" + this.levels[i] + ":'" + this.strings[i] + "'] ");
  //            }
  //            System.out.println();
  //            System.out.println();
  //        }
  //
  //    public static void main(String [] args) throws Exception {
  //
  //        new PreparedVariableResolver("&{../../blah}").dump();
  //        new PreparedVariableResolver("{t1}tt{t2}x").dump();
  //        new PreparedVariableResolver("\\{t1}tt{t2}xx").dump();
  //        new PreparedVariableResolver("{t1}tt\\{t2}xx").dump();
  //        new PreparedVariableResolver("{t1}tt{t2}xx").dump();
  //        new PreparedVariableResolver("xx{../t1}{../../../t2}zz").dump();
  //        new PreparedVariableResolver("xx{../t1}\\{../../../t2}zz").dump();
  //
  //    }
  
  }
  
  
  
  1.1                  xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/variables/VariableResolver.java
  
  Index: VariableResolver.java
  ===================================================================
  /*
  
   ============================================================================
                     The Apache Software License, Version 1.1
   ============================================================================
  
   Copyright (C) 1999-2002 The Apache Software Foundation. All rights reserved.
  
   Redistribution and use in source and binary forms, with or without modifica-
   tion, are permitted provided that the following conditions are met:
  
   1. Redistributions of  source code must  retain the above copyright  notice,
      this list of conditions and the following disclaimer.
  
   2. Redistributions in binary form must reproduce the above copyright notice,
      this list of conditions and the following disclaimer in the documentation
      and/or other materials provided with the distribution.
  
   3. The end-user documentation included with the redistribution, if any, must
      include  the following  acknowledgment:  "This product includes  software
      developed  by the  Apache Software Foundation  (http://www.apache.org/)."
      Alternately, this  acknowledgment may  appear in the software itself,  if
      and wherever such third-party acknowledgments normally appear.
  
   4. The names "Apache Cocoon" and  "Apache Software Foundation" must  not  be
      used to  endorse or promote  products derived from  this software without
      prior written permission. For written permission, please contact
      apache@apache.org.
  
   5. Products  derived from this software may not  be called "Apache", nor may
      "Apache" appear  in their name,  without prior written permission  of the
      Apache Software Foundation.
  
   THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
   INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
   FITNESS  FOR A PARTICULAR  PURPOSE ARE  DISCLAIMED.  IN NO  EVENT SHALL  THE
   APACHE SOFTWARE  FOUNDATION  OR ITS CONTRIBUTORS  BE LIABLE FOR  ANY DIRECT,
   INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR CONSEQUENTIAL  DAMAGES (INCLU-
   DING, BUT NOT LIMITED TO, PROCUREMENT  OF SUBSTITUTE GOODS OR SERVICES; LOSS
   OF USE, DATA, OR  PROFITS; OR BUSINESS  INTERRUPTION)  HOWEVER CAUSED AND ON
   ANY  THEORY OF LIABILITY,  WHETHER  IN CONTRACT,  STRICT LIABILITY,  OR TORT
   (INCLUDING  NEGLIGENCE OR  OTHERWISE) ARISING IN  ANY WAY OUT OF THE  USE OF
   THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
  
   This software  consists of voluntary contributions made  by many individuals
   on  behalf of the Apache Software  Foundation and was  originally created by
   Stefano Mazzocchi  <st...@apache.org>. For more  information on the Apache
   Software Foundation, please see <http://www.apache.org/>.
  
  */
  package org.apache.cocoon.components.treeprocessor.variables;
  
  import org.apache.avalon.framework.parameters.Parameters;
  import org.apache.cocoon.sitemap.PatternException;
  
  import java.util.Collections;
  import java.util.Iterator;
  import java.util.HashMap;
  import java.util.List;
  import java.util.Map;
  
  /**
   * Utility class for handling {...} pattern substitutions in sitemap statements.
   *
   * @author <a href="mailto:sylvain@apache.org">Sylvain Wallez</a>
   * @version CVS $Id: VariableResolver.java,v 1.1 2002/06/24 20:32:20 sylvain Exp $
   */
  
  public abstract class VariableResolver {
      
      public static final Map EMPTY_MAP = Collections.unmodifiableMap(new java.util.HashMap(0));
  
      /**
       * Resolve all {...} patterns using the values given in the list of maps and the object model.
       */
      public abstract String resolve(List mapStack, Map objectModel) throws PatternException;
  
      /**
       * Build a <code>Parameters</code> object from a Map of named <code>VariableResolver</code>s and
       * a list of Maps used for resolution.
       *
       * @return a fully resolved <code>Parameters</code>.
       */
      public static Parameters buildParameters(Map expressions, List mapStack, Map objectModel) throws PatternException {
          if (expressions == null || expressions.size() == 0) {
              return Parameters.EMPTY_PARAMETERS;
          }
  
          Parameters result = new Parameters();
  
          Iterator iter = expressions.entrySet().iterator();
          while (iter.hasNext()) {
              Map.Entry entry = (Map.Entry)iter.next();
              result.setParameter(
                  ((VariableResolver)entry.getKey()).resolve(mapStack, objectModel),
                  ((VariableResolver)entry.getValue()).resolve(mapStack, objectModel)
              );
          }
  
          return result;
      }
  
      /**
       * Build a <code>Map</code> from a Map of named <code>ListOfMapResolver</code>s and
       * a list of Maps used for resolution.
       *
       * @return a fully resolved <code>Map</code>.
       */
      public static Map buildMap(Map expressions, List mapStack, Map objectModel) throws PatternException {
          int size;
          if (expressions == null || (size = expressions.size()) == 0) {
              return EMPTY_MAP;
          }
  
          Map result = new HashMap(size);
  
          Iterator iter = expressions.entrySet().iterator();
          while (iter.hasNext()) {
              Map.Entry entry = (Map.Entry)iter.next();
              result.put(
                  ((VariableResolver)entry.getKey()).resolve(mapStack, objectModel),
                  ((VariableResolver)entry.getValue()).resolve(mapStack, objectModel)
              );
          }
  
          return result;
      }
  
  //    /**
  //     * Release a <code>Map</code> of expressions.
  //     */
  //    public static void release(Map expressions) {
  //        Iterator iter = expressions.entrySet().iterator();
  //        while (iter.hasNext()) {
  //            Map.Entry entry = (Map.Entry)iter.next();
  //            ((VariableResolver)entry.getKey()).release();
  //            ((VariableResolver)entry.getValue()).release();
  //        }
  //    }
  }
  
  
  1.1                  xml-cocoon2/src/java/org/apache/cocoon/components/treeprocessor/variables/VariableResolverFactory.java
  
  Index: VariableResolverFactory.java
  ===================================================================
  /*
  
   ============================================================================
                     The Apache Software License, Version 1.1
   ============================================================================
  
   Copyright (C) 1999-2002 The Apache Software Foundation. All rights reserved.
  
   Redistribution and use in source and binary forms, with or without modifica-
   tion, are permitted provided that the following conditions are met:
  
   1. Redistributions of  source code must  retain the above copyright  notice,
      this list of conditions and the following disclaimer.
  
   2. Redistributions in binary form must reproduce the above copyright notice,
      this list of conditions and the following disclaimer in the documentation
      and/or other materials provided with the distribution.
  
   3. The end-user documentation included with the redistribution, if any, must
      include  the following  acknowledgment:  "This product includes  software
      developed  by the  Apache Software Foundation  (http://www.apache.org/)."
      Alternately, this  acknowledgment may  appear in the software itself,  if
      and wherever such third-party acknowledgments normally appear.
  
   4. The names "Apache Cocoon" and  "Apache Software Foundation" must  not  be
      used to  endorse or promote  products derived from  this software without
      prior written permission. For written permission, please contact
      apache@apache.org.
  
   5. Products  derived from this software may not  be called "Apache", nor may
      "Apache" appear  in their name,  without prior written permission  of the
      Apache Software Foundation.
  
   THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
   INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
   FITNESS  FOR A PARTICULAR  PURPOSE ARE  DISCLAIMED.  IN NO  EVENT SHALL  THE
   APACHE SOFTWARE  FOUNDATION  OR ITS CONTRIBUTORS  BE LIABLE FOR  ANY DIRECT,
   INDIRECT, INCIDENTAL, SPECIAL,  EXEMPLARY, OR CONSEQUENTIAL  DAMAGES (INCLU-
   DING, BUT NOT LIMITED TO, PROCUREMENT  OF SUBSTITUTE GOODS OR SERVICES; LOSS
   OF USE, DATA, OR  PROFITS; OR BUSINESS  INTERRUPTION)  HOWEVER CAUSED AND ON
   ANY  THEORY OF LIABILITY,  WHETHER  IN CONTRACT,  STRICT LIABILITY,  OR TORT
   (INCLUDING  NEGLIGENCE OR  OTHERWISE) ARISING IN  ANY WAY OUT OF THE  USE OF
   THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
  
   This software  consists of voluntary contributions made  by many individuals
   on  behalf of the Apache Software  Foundation and was  originally created by
   Stefano Mazzocchi  <st...@apache.org>. For more  information on the Apache
   Software Foundation, please see <http://www.apache.org/>.
  
  */
  package org.apache.cocoon.components.treeprocessor.variables;
  
  import org.apache.avalon.framework.component.ComponentManager;
  
  import org.apache.cocoon.sitemap.PatternException;
  
  import java.util.Collections;
  import java.util.Iterator;
  import java.util.HashMap;
  import java.util.List;
  import java.util.Map;
  
  public class VariableResolverFactory {
      
      private static ThreadLocal disposableCollector = new ThreadLocal();
      
      /**
       * Set the thread-local list where all created resolvers that need to be
       * disposed will be collected.
       * <p>
       * The purpose of collecting resolvers is to avoid manual release (or lack thereof)
       * that requires most <code>ProcessingNodes</code> to implement <code>Disposable</code>.
       */
      public static void setDisposableCollector(List collector) {
          disposableCollector.set(collector);
      }
  
      /**
       * Does an expression need resolving (i.e. contain {...} patterns) ?
       */
      public static boolean needsResolve(String expression) {
          if (expression == null || expression.length() == 0) {
              return false;
          }
  
          // Is the first char a '{' ?
          if (expression.charAt(0) == '{') {
              return true;
          }
  
          if (expression.length() < 2) {
              return false;
          }
  
          // Is there any unescaped '{' ?
          int pos = 1;
          while ( (pos = expression.indexOf('{', pos)) != -1) {
              // Found a '{' : is it escaped ?
              if (expression.charAt(pos - 1) != '\\') {
                  // No : need to resolve
                  return true;
              }
              pos++;
          }
          // Nothing found...
          return false;
      }
      
      /**
       * Unescape an expression that doesn't need to be resolved, but may contain
       * escaped '{' characters.
       *
       * @param expression the expression to unescape.
       * @return the unescaped result, or <code>expression</code> if unescaping isn't necessary.
       */
      public static String unescape(String expression) {
          // Does it need escaping ?
          if (expression == null || expression.indexOf("\\{") == -1) {
              return expression;
          }
  
          StringBuffer buf = new StringBuffer();
          for (int i = 0; i < expression.length(); i++) {
              char ch = expression.charAt(i);
              if (ch != '\\' || i >= (expression.length() - 1) || expression.charAt(i+1) != '{') {
                  buf.append(ch);
              }
          }
  
          return buf.toString();
      }
  
      /**
       * Get a resolver for a given expression. Chooses the most efficient implementation
       * depending on <code>expression</code>.
       */
      public static VariableResolver getResolver(String expression, ComponentManager manager) throws PatternException {
          if (needsResolve(expression)) {
              VariableResolver resolver = new PreparedVariableResolver(expression, manager);
              List collector = (List)disposableCollector.get();
              if (collector != null)
                  collector.add(resolver);
              
              return resolver;
              
          } else {
              return new NOPVariableResolver(expression);
              
          }
      }
  
  
  }
  
  
  
  
  

----------------------------------------------------------------------
In case of troubles, e-mail:     webmaster@xml.apache.org
To unsubscribe, e-mail:          cocoon-cvs-unsubscribe@xml.apache.org
For additional commands, e-mail: cocoon-cvs-help@xml.apache.org