You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@shindig.apache.org by aw...@apache.org on 2009/05/29 23:16:13 UTC

svn commit: r780108 - in /incubator/shindig/trunk: config/ java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/ java/gadgets/src/main/resources/org/apache/shindig/gadgets/templa...

Author: awiner
Date: Fri May 29 21:15:53 2009
New Revision: 780108

URL: http://svn.apache.org/viewvc?rev=780108&view=rev
Log:
Remainder of Lev Epshteyn patch from http://codereview.appspot.com/53052, heavily modified.
- Include per-tag and per-library resources only when used
- Support per-container OSML definition
- Remove unused NameTagHander class

Added:
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java   (with props)
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java   (with props)
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java   (with props)
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java   (contents, props changed)
      - copied, changed from r779276, incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java   (contents, props changed)
      - copied, changed from r779276, incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/TemplateLibraryTest.java
    incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml   (with props)
Removed:
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NameTagHandler.java
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/TemplateLibraryTest.java
Modified:
    incubator/shindig/trunk/config/container.js
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/TemplateRewriter.java
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateContext.java
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibraryFactory.java
    incubator/shindig/trunk/java/gadgets/src/main/resources/org/apache/shindig/gadgets/templates/OSML_library.xml
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/TemplateRewriterTest.java
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/DefaultTemplateProcessorTest.java
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/RenderTagHandlerTest.java

Modified: incubator/shindig/trunk/config/container.js
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/config/container.js?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/config/container.js (original)
+++ incubator/shindig/trunk/config/container.js Fri May 29 21:15:53 2009
@@ -147,5 +147,10 @@
   "osapi" : {
     // The endpoints to query for available JSONRPC/REST services
     "endPoints" : [ "http://%host%/social/rpc", "http://%host%/gadgets/api/rpc" ]                   
+  },
+  "osml": {
+    // OSML library resource.  Can be set to null or the empty string to disable OSML
+    // for a container.
+    "library": "org/apache/shindig/gadgets/templates/OSML_library.xml"
   }
 }}

Modified: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/TemplateRewriter.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/TemplateRewriter.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/TemplateRewriter.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/TemplateRewriter.java Fri May 29 21:15:53 2009
@@ -17,13 +17,14 @@
  */
 package org.apache.shindig.gadgets.rewrite;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.shindig.common.JsonSerializer;
 import org.apache.shindig.common.uri.Uri;
 import org.apache.shindig.common.util.ResourceLoader;
 import org.apache.shindig.common.xml.DomUtil;
 import org.apache.shindig.common.xml.XmlException;
 import org.apache.shindig.common.xml.XmlUtil;
+import org.apache.shindig.config.ContainerConfig;
 import org.apache.shindig.expressions.Expressions;
 import org.apache.shindig.gadgets.Gadget;
 import org.apache.shindig.gadgets.GadgetContext;
@@ -35,6 +36,7 @@
 import org.apache.shindig.gadgets.templates.CompositeTagRegistry;
 import org.apache.shindig.gadgets.templates.DefaultTagRegistry;
 import org.apache.shindig.gadgets.templates.MessageELResolver;
+import org.apache.shindig.gadgets.templates.NullTemplateLibrary;
 import org.apache.shindig.gadgets.templates.TagHandler;
 import org.apache.shindig.gadgets.templates.TagRegistry;
 import org.apache.shindig.gadgets.templates.TemplateBasedTagHandler;
@@ -43,22 +45,28 @@
 import org.apache.shindig.gadgets.templates.TemplateLibraryFactory;
 import org.apache.shindig.gadgets.templates.TemplateParserException;
 import org.apache.shindig.gadgets.templates.TemplateProcessor;
+import org.apache.shindig.gadgets.templates.TemplateResource;
+import org.apache.shindig.gadgets.templates.XmlTemplateLibrary;
 import org.w3c.dom.DocumentFragment;
 import org.w3c.dom.Element;
+import org.w3c.dom.Node;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.StringTokenizer;
+import java.util.concurrent.ConcurrentMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.MapMaker;
 import com.google.inject.Inject;
 import com.google.inject.Provider;
 
@@ -80,6 +88,7 @@
   static final Object REQUIRE_LIBRARY_PARAM = "requireLibrary";
 
   static private final Logger logger = Logger.getLogger(TemplateRewriter.class.getName());
+  
   /**
    * Provider of the processor.  TemplateRewriters are stateless and multithreaded,
    * processors are not.
@@ -89,26 +98,44 @@
   private final Expressions expressions;
   private final TagRegistry baseTagRegistry;
   private final TemplateLibraryFactory libraryFactory;
+  private final ContainerConfig config;
   
-  private final TemplateLibrary osmlLibrary;
+  private final ConcurrentMap<String, TemplateLibrary> osmlLibraryCache = 
+    new MapMaker().makeComputingMap(
+        new Function<String, TemplateLibrary>() {
+          public TemplateLibrary apply(String resourceName) {
+            return loadTrustedLibrary(resourceName);
+          }
+        });
 
   @Inject
   public TemplateRewriter(Provider<TemplateProcessor> processor,
       MessageBundleFactory messageBundleFactory, Expressions expressions, 
-      TagRegistry baseTagRegistry, TemplateLibraryFactory libraryFactory) {
+      TagRegistry baseTagRegistry, TemplateLibraryFactory libraryFactory,
+      ContainerConfig config) {
     this.processor = processor;
     this.messageBundleFactory = messageBundleFactory;
     this.expressions = expressions;
     this.baseTagRegistry = baseTagRegistry;
     this.libraryFactory = libraryFactory;
-    this.osmlLibrary = loadOsmlLibrary();
+    this.config = config;
   }
 
-  private TemplateLibrary loadOsmlLibrary() {
+  private TemplateLibrary getOsmlLibrary(Gadget gadget) {
+    String library = config.getString(gadget.getContext().getContainer(),
+        "${Cur['gadgets.features'].osml.library}");
+    if (StringUtils.isEmpty(library)) {
+      return NullTemplateLibrary.INSTANCE;
+    }
+    
+    return osmlLibraryCache.get(library);
+  }
+  
+  static private TemplateLibrary loadTrustedLibrary(String resource) {
     try {
-      String content = ResourceLoader.getContent(
-          "org/apache/shindig/gadgets/templates/OSML_library.xml");              
-      return new TemplateLibrary(Uri.parse("#OSML"), XmlUtil.parse(content), true);
+      String content = ResourceLoader.getContent(resource);
+      return new XmlTemplateLibrary(Uri.parse("#OSML"), XmlUtil.parse(content), 
+          content, true);
     } catch (IOException ioe) {
       logger.log(Level.WARNING, null, ioe);
     } catch (XmlException xe) {
@@ -116,7 +143,8 @@
     } catch (GadgetException tpe) {
       logger.log(Level.WARNING, null, tpe);
     }
-    return null;
+
+    return NullTemplateLibrary.INSTANCE;
   }
   
   public void rewrite(Gadget gadget, MutableContent content) {
@@ -146,16 +174,17 @@
   private void rewriteImpl(Gadget gadget, Feature f, MutableContent content)
       throws GadgetException {   
     List<TagRegistry> registries = Lists.newArrayList();
-
-    Element head = (Element) DomUtil.getFirstNamedChildNode(
-        content.getDocument().getDocumentElement(), "head");
+    List<TemplateLibrary> libraries = Lists.newArrayList();
+   
+    // TODO: Add View-specific library as Priority 0
     
+    // Built-in Java-based tags - Priority 1
     registries.add(baseTagRegistry);
     
-    if (osmlLibrary != null) {
-      registries.add(osmlLibrary.getTagRegistry());
-      injectTemplateLibrary(osmlLibrary, head);
-    }
+    TemplateLibrary osmlLibrary = getOsmlLibrary(gadget);
+    // OSML Built-in tags - Priority 2
+    registries.add(osmlLibrary.getTagRegistry());
+    libraries.add(osmlLibrary);
 
     List<Element> templates = ImmutableList.copyOf(
         Iterables.filter(
@@ -166,16 +195,63 @@
               }
             }));
     
-    loadTemplateLibraries(gadget.getContext(), f, registries, head);
+    // User-defined custom tags - Priority 3
     registries.add(registerCustomTags(templates));
     
+    // User-defined libraries - Priority 4
+    loadTemplateLibraries(gadget.getContext(), f, registries, libraries);
+    
     TagRegistry registry = new CompositeTagRegistry(registries);
     
-    processInlineTemplates(gadget, content, templates, registry);
+    TemplateContext templateContext = new TemplateContext(gadget, content.getPipelinedData());    
+    boolean needsFeature = executeTemplates(templateContext, content, templates, registry);
+
+    Element head = (Element) DomUtil.getFirstNamedChildNode(
+        content.getDocument().getDocumentElement(), "head");
+    postProcess(templateContext, needsFeature, head, templates, libraries);
+  }
+
+  /**
+   * Post-processes the gadget content after rendering templates.
+   * 
+   * @param templateContext TemplateContext to operate on
+   * @param needsFeature Should the templates feature be made available to 
+   * client?
+   * @param head Head element of the gadget's document
+   * @param libraries Keeps track of all libraries, and which got used
+   * @param allTemplates A list of all the template nodes
+   * @param libraries A list of all registered libraries
+   */
+  private void postProcess(TemplateContext templateContext, boolean needsFeature, Element head,
+      List<Element> allTemplates, List<TemplateLibrary> libraries) {
+    // Inject all the needed library assets.
+    // TODO: inject library assets that aren't used on the server, but will
+    // be needed on the client
+    for (TemplateResource resource : templateContext.getResources()) {
+      injectTemplateLibraryAssets(resource, head);
+    }
+
+    // If we don't need the feature, remove it and all templates from the gadget
+    if (!needsFeature) {
+      templateContext.getGadget().removeFeature("opensocial-templates");
+      for (Element template : allTemplates) {
+        Node parent = template.getParentNode();
+        if (parent != null) {
+          parent.removeChild(template);
+        }
+      }
+    } else {
+      // If the feature is to be kept, inject the libraries.
+      // Library assets will be generated on the client.
+      // TODO: only inject the templates, not the full scripts/styles
+      for (TemplateLibrary library : libraries) {
+        injectTemplateLibrary(library, head);
+      }
+    }
   }
 
   private void loadTemplateLibraries(GadgetContext context,
-      Feature f, List<TagRegistry> registries, Element head)  throws GadgetException {
+      Feature f, List<TagRegistry> registries, List<TemplateLibrary> libraries)  throws GadgetException {
     // TODO: Support multiple values when Shindig does
     String url = f.getParams().get(REQUIRE_LIBRARY_PARAM);
     if (url != null) {
@@ -184,12 +260,8 @@
       
       try {
         TemplateLibrary library = libraryFactory.loadTemplateLibrary(context, uri);
-        String script = library.getJavaScript();
-        
-        // TODO: Only inject if used?
-        injectTemplateLibrary(library, head);
-        
         registries.add(library.getTagRegistry());
+        libraries.add(library);
       } catch (TemplateParserException te) {
         // Suppress exceptions due to malformed template libraries
         logger.log(Level.WARNING, null, te);
@@ -197,30 +269,48 @@
     }
   }
   
-  private void injectTemplateLibrary(TemplateLibrary library, Element head) {  
-    // Append any needed Javascript
-    String script = library.getJavaScript();
-    if (!StringUtils.isEmpty(script)) {
+  private void injectTemplateLibraryAssets(TemplateResource resource, Element head) {
+    Element contentElement;
+    switch (resource.getType()) {
+      case JAVASCRIPT:
+        contentElement = head.getOwnerDocument().createElement("script");
+        contentElement.setAttribute("type", "text/javascript");
+        break;
+      case STYLE:
+        contentElement = head.getOwnerDocument().createElement("style");
+        contentElement.setAttribute("type", "text/css");
+        break;
+      default:
+        throw new IllegalStateException("Unhandled type");  
+    }
+
+    if (resource.isSafe()) {
+      SanitizingGadgetRewriter.bypassSanitization(contentElement, false);
+    }
+    contentElement.setTextContent(resource.getContent());
+    head.appendChild(contentElement);    
+  }
+  
+  private void injectTemplateLibrary(TemplateLibrary library, Element head) {
+    try {
+      String libraryContent = library.serialize();
+      if (StringUtils.isEmpty(libraryContent)) {
+        return;
+      }
+      
       Element scriptElement = head.getOwnerDocument().createElement("script");
       scriptElement.setAttribute("type", "text/javascript");
-      scriptElement.setTextContent(script);
-      if (library.isSafe()) {
-        SanitizingGadgetRewriter.bypassSanitization(scriptElement, false);
-      }
+      StringBuilder buffer = new StringBuilder();
+      buffer.append("opensocial.template.Loader.loadContent(");
+      JsonSerializer.appendString(buffer, library.serialize());
+      buffer.append(",");
+      JsonSerializer.appendString(buffer, library.getLibraryUri().toString());
+      buffer.append(");");       
+      scriptElement.setTextContent(buffer.toString());
       head.appendChild(scriptElement);
+    } catch (IOException ioe) {
+      // This should never happen.
     }
-    
-    // Append any needed CSS
-    String style = library.getStyle();
-    if (!StringUtils.isEmpty(style)) {
-      Element styleElement = head.getOwnerDocument().createElement("style");
-      styleElement.setAttribute("type", "text/css");
-      styleElement.setTextContent(style);
-      if (library.isSafe()) {
-        SanitizingGadgetRewriter.bypassSanitization(styleElement, false);
-      }
-      head.appendChild(styleElement);
-    } 
   }
   
   /**
@@ -249,7 +339,11 @@
     return new DefaultTagRegistry(handlers.build());
   }
   
-  private void processInlineTemplates(Gadget gadget, MutableContent content,
+  /**
+   * Processes and renders inline templates.
+   * @return Do we think the templates feature is still needed on the client?
+   */
+  private boolean executeTemplates(TemplateContext templateContext, MutableContent content,
       List<Element> allTemplates, TagRegistry registry) throws GadgetException {
     Map<String, Object> pipelinedData = content.getPipelinedData();
 
@@ -271,7 +365,7 @@
     }
     
     if (!templates.isEmpty()) {
-      TemplateContext templateContext = new TemplateContext(gadget, pipelinedData);
+      Gadget gadget = templateContext.getGadget();
       
       MessageBundle bundle = messageBundleFactory.getBundle(gadget.getSpec(),
           gadget.getContext().getLocale(), gadget.getContext().getIgnoreCache());
@@ -281,6 +375,7 @@
         DocumentFragment result = processor.get().processTemplate(
             template, templateContext, messageELResolver, registry);
         template.getParentNode().insertBefore(result, template);
+        // TODO: sanitized renders should ignore this value
         if ("true".equals(template.getAttribute("autoUpdate"))) {
           // autoUpdate requires client-side processing.
           // TODO: give client-side processing some hope of finding the pre-rendered content
@@ -288,19 +383,12 @@
         } else {
           template.getParentNode().removeChild(template);
         }
-      }
-  
+      } 
       MutableContent.notifyEdit(content.getDocument());
-    }
-    
-    // Remove the opensocial-templates feature if we've fully processed all
-    // inline templates.
-    // TODO: remove inline custom tags as well.
-    if (!needsFeature) {
-      gadget.removeFeature("opensocial-templates");
-    }
+    } 
+    return needsFeature;
   }
-
+  
   /**
    * Checks that all the required data is available at rewriting time.
    * @param requiredData A string of comma-separated data set names

Added: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java?rev=780108&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java Fri May 29 21:15:53 2009
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.shindig.gadgets.templates;
+
+import org.apache.shindig.common.uri.Uri;
+
+/**
+ * A default implementation of TemplateLibrary which serves as a wrapper for
+ * an existing TagRegistry.
+ */
+public class DefaultTemplateLibrary implements TemplateLibrary {
+
+  private final TagRegistry registry;
+  
+  public DefaultTemplateLibrary(TagRegistry registry) {
+    this.registry = registry;
+  }
+  
+  public String getJavaScript() {
+    return null;
+  }
+
+  public Uri getLibraryUri() {
+    return null;
+  }
+
+  public String getStyle() {
+    return null;
+  }
+
+  public TagRegistry getTagRegistry() {
+    return registry;
+  }
+
+  public boolean isSafe() {
+    return false;
+  }
+
+  public String serialize() {
+    return null;
+  }
+
+}

Propchange: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/DefaultTemplateLibrary.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java?rev=780108&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java Fri May 29 21:15:53 2009
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.shindig.gadgets.templates;
+
+import org.apache.shindig.common.uri.Uri;
+
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Null object implementation of TemplateLibrary.
+ */
+public class NullTemplateLibrary implements TemplateLibrary {
+  public static final TemplateLibrary INSTANCE = new NullTemplateLibrary();
+
+  private final TagRegistry registry = new DefaultTagRegistry(ImmutableSet.<TagHandler>of());
+  
+  private NullTemplateLibrary() {  
+  }
+  
+  public Uri getLibraryUri() {
+    return null;
+  }
+
+  public TagRegistry getTagRegistry() {
+    return registry;
+  }
+
+  public boolean isSafe() {
+    return false;
+  }
+
+  public String serialize() {
+    return null;
+  }
+}

Propchange: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/NullTemplateLibrary.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateContext.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateContext.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateContext.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateContext.java Fri May 29 21:15:53 2009
@@ -21,9 +21,11 @@
 import org.apache.shindig.gadgets.Gadget;
 import org.w3c.dom.Node;
 
+import java.util.Collection;
 import java.util.Map;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 
 /**
  * Context for processing a single template.
@@ -37,6 +39,7 @@
   private Map<String, ? extends Object> context = ImmutableMap.of();
   private Map<String, Object> myMap = null;
   private Node templateRoot;
+  private Map<Object, TemplateResource> resources = Maps.newLinkedHashMap();
   
   public TemplateContext(Gadget gadget, Map<String, ? extends Object> top) {
     this.gadget = gadget;
@@ -91,4 +94,14 @@
   public Node getTemplateRoot() {
     return this.templateRoot;
   }
+  
+  public void addResource(Object key, TemplateResource resource) {
+    if (!resources.containsKey(key)) {
+      resources.put(key, resource);
+    }
+  }
+  
+  public Collection<TemplateResource> getResources() {
+    return resources.values();
+  }
 }

Modified: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java Fri May 29 21:15:53 2009
@@ -19,204 +19,22 @@
 package org.apache.shindig.gadgets.templates;
 
 import org.apache.shindig.common.uri.Uri;
-import org.apache.shindig.common.xml.DomUtil;
-import org.apache.shindig.gadgets.GadgetException;
-import org.w3c.dom.Attr;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.ImmutableSet.Builder;
 
 /**
- * An Object representing a Library of Template-based custom OSML tags.
+ * A Template Library is a collection of tag handlers, and any necessary
+ * assets (CSS and Javascript).
  */
-public class TemplateLibrary {
-
-  public static final String TAG_ATTRIBUTE = "tag";
-  public static final String NAMESPACE_TAG = "Namespace";
-  public static final String TEMPLATE_TAG = "Template";
-  public static final String STYLE_TAG = "Style";
-  public static final String JAVASCRIPT_TAG = "JavaScript";
-  public static final String TEMPLATEDEF_TAG = "TemplateDef";
-  
-  private final Uri libraryUri;
-  private final boolean safe;
-  private TagRegistry registry;
-  private String nsPrefix;
-  private String nsUri;
-  private String style;
-  private String javaScript;
-
-  /**
-   * @param uri URI of the template library
-   * @param root Element representing the Templates tag of this library
-   */
-  public TemplateLibrary(Uri uri, Element root) throws GadgetException, TemplateParserException {
-    this(uri, root, false);
-  }
-
-  /**
-   * @param uri URI of the template library
-   * @param root Element representing the Templates tag of this library
-   * @param safe Is this library exempt from being sanitized?
-   */
-  public TemplateLibrary(Uri uri, Element root, boolean safe) 
-      throws GadgetException, TemplateParserException {
-    libraryUri = uri;
-    registry = new DefaultTagRegistry(parseLibraryDocument(root));
-    this.safe = safe;
-  }
+public interface TemplateLibrary {
 
   /**
    * @return a registry of tags in this library.
    */
-  public TagRegistry getTagRegistry() {
-    return registry;
-  }
-  
-  /**
-   * @return the URI from which the library was loaded.  (This is not the
-   * namespace of tags in the library.)
-   */
-  public Uri getLibraryUri() {
-    return libraryUri;
-  }
-  
-  /**
-   * @return the concatenated contents of Style elements for the library.
-   */
-  public String getStyle() {
-    return style;
-  }
+  public TagRegistry getTagRegistry();
   
-  /**
-   * @return this library is safe and its content doesn't need to be sanitized. 
-   */
-  public boolean isSafe() {
-    return safe;
-  }
-  
-  /**
-   * TODO: Minify javascript (here or elsewhere)
-   * @return the concatenated contents of JavaScript elements for the library.
-   */
-  public String getJavaScript() {
-    return javaScript;
-  }
+  public Uri getLibraryUri();
+
+  public boolean isSafe();
   
-  private Set<TagHandler> parseLibraryDocument(Element root)
-      throws GadgetException, TemplateParserException {
-    ImmutableSet.Builder<TagHandler> handlers = ImmutableSet.builder();
-    
-    NodeList nodes = root.getChildNodes();
-    for (int i = 0; i < nodes.getLength(); i++) {
-      Node node = nodes.item(i);
-      if (!(node instanceof Element)) {
-        continue;
-      }
-      
-      Element element = (Element) node;
-      if (NAMESPACE_TAG.equals(element.getLocalName())) {
-        processNamespace(element);
-      } else if (STYLE_TAG.equals(element.getLocalName())) {
-        processStyle(element);
-      } else if (JAVASCRIPT_TAG.equals(element.getLocalName())) {
-        processJavaScript(element);
-      } else if (TEMPLATE_TAG.equals(element.getLocalName())) {
-        processTemplate(handlers, element);
-      } else if (TEMPLATEDEF_TAG.equals(element.getLocalName())) {
-        processTemplateDef(handlers, element);
-      }
-    }
-    
-    return handlers.build();
-  }
+  public String serialize();
   
-  private void processTemplateDef(Builder<TagHandler> handlers, Element defElement)
-      throws TemplateParserException {
-    Attr tagAttribute = defElement.getAttributeNode(TAG_ATTRIBUTE);
-    if (tagAttribute == null) {
-      throw new TemplateParserException("Missing tag attribute on TemplateDef");
-    }
-
-    Element templateElement = (Element) DomUtil.getFirstNamedChildNode(defElement, TEMPLATE_TAG);
-    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement);
-    if (handler != null) {
-      handlers.add(handler);
-    }
-    
-    Element scriptElement = (Element) DomUtil.getFirstNamedChildNode(defElement, JAVASCRIPT_TAG);
-    if (scriptElement != null) {
-      processJavaScript(scriptElement);
-    }
-    
-    Element styleElement = (Element) DomUtil.getFirstNamedChildNode(defElement, STYLE_TAG);
-    if (styleElement != null) {
-      processStyle(styleElement);
-    }
-  }
-
-  private void processTemplate(Builder<TagHandler> handlers, Element templateElement)
-      throws TemplateParserException {
-    Attr tagAttribute = templateElement.getAttributeNode(TAG_ATTRIBUTE);
-    if (tagAttribute == null) {
-      throw new TemplateParserException("Missing tag attribute on Template");
-    }
-    
-    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement);
-    if (handler != null) {
-      handlers.add(handler);
-    }
-  }
-
-  private void processStyle(Element element) {
-    if (style == null) {
-      style = element.getTextContent();
-    } else {
-      style = style + "\n" + element.getTextContent();
-    }
-  }
-
-  private void processJavaScript(Element element) {
-    if (javaScript == null) {
-      javaScript = element.getTextContent();
-    } else {
-      javaScript = javaScript + "\n" + element.getTextContent();
-    }
-  }
-
-  private void processNamespace(Element namespaceNode) throws TemplateParserException {
-    if ((nsPrefix != null) || (nsUri != null)) {
-      throw new TemplateParserException("Duplicate Namespace elements");
-    }
-    
-    nsPrefix = namespaceNode.getAttribute("prefix");
-    if ("".equals(nsPrefix)) {
-      throw new TemplateParserException("Missing prefix attribute on Namespace");
-    }
-    
-    nsUri = namespaceNode.getAttribute("url");
-    if ("".equals(nsUri)) {
-      throw new TemplateParserException("Missing url attribute on Namespace");
-    }
-  }
-
-  private TagHandler createHandler(String tagName, Element template)
-      throws TemplateParserException {
-    String [] nameParts = tagName.split(":");
-    // At this time, we only support namespaced tags
-    if (nameParts.length != 2) {
-      return null;
-    }
-    String namespaceUri = template.lookupNamespaceURI(nameParts[0]);
-    if (!nsPrefix.equals(nameParts[0]) || !nsUri.equals(namespaceUri)) {
-      throw new TemplateParserException(
-          "Can't create tags in undeclared namespace: " + nameParts[0]);
-    }
-    return new TemplateBasedTagHandler(template, namespaceUri, nameParts[1]);
-  }
 }

Modified: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibraryFactory.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibraryFactory.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibraryFactory.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibraryFactory.java Fri May 29 21:15:53 2009
@@ -81,7 +81,7 @@
         }
       }
 
-      return new TemplateLibrary(uri, element);
+      return new XmlTemplateLibrary(uri, element, content);
     } catch (XmlException e) {
       throw new GadgetException(GadgetException.Code.MALFORMED_XML_DOCUMENT, e);
     }

Added: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java?rev=780108&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java Fri May 29 21:15:53 2009
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.shindig.gadgets.templates;
+
+/**
+ * Encapsulation of a single resource imported by a library for template
+ * execution.
+ */
+public final class TemplateResource {
+  private final String content;
+  private final Type type;
+  private final boolean isSafe;
+
+  public enum Type { JAVASCRIPT, STYLE };
+  
+  /**
+   * Create a Javascript resource.
+   * @param javascript the script content
+   * @param library the library that is the source of the script
+   */
+  public static TemplateResource newJavascriptResource(String javascript, TemplateLibrary library) {
+    return new TemplateResource(javascript, Type.JAVASCRIPT, library.isSafe());
+  }
+  
+  /**
+   * Create a CSS resource.
+   * @param style the CSS content
+   * @param library the library that is the source of the content
+   */
+  public static TemplateResource newStyleResource(String style, TemplateLibrary library) {
+    return new TemplateResource(style, Type.STYLE, library.isSafe());
+  }
+
+  private TemplateResource(String content, Type type, boolean isSafe) {
+    this.content = content;
+    this.type = type;
+    this.isSafe = isSafe;
+  }
+
+  public String getContent() {
+    return content;
+  }
+
+  public Type getType() {
+    return type;
+  }
+
+  public boolean isSafe() {
+    return isSafe;
+  }
+  
+  @Override
+  public String toString() {
+    return "<" + type + ">" + content + "</" + type + ">"; 
+  }
+}

Propchange: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateResource.java
------------------------------------------------------------------------------
    svn:eol-style = native

Copied: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java (from r779276, incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java)
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java?p2=incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java&p1=incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java&r1=779276&r2=780108&rev=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/TemplateLibrary.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java Fri May 29 21:15:53 2009
@@ -34,8 +34,8 @@
 /**
  * An Object representing a Library of Template-based custom OSML tags.
  */
-public class TemplateLibrary {
-
+public class XmlTemplateLibrary implements TemplateLibrary { 
+   
   public static final String TAG_ATTRIBUTE = "tag";
   public static final String NAMESPACE_TAG = "Namespace";
   public static final String TEMPLATE_TAG = "Template";
@@ -44,19 +44,22 @@
   public static final String TEMPLATEDEF_TAG = "TemplateDef";
   
   private final Uri libraryUri;
+  private final String source;
   private final boolean safe;
   private TagRegistry registry;
   private String nsPrefix;
   private String nsUri;
   private String style;
   private String javaScript;
+  private Set<TemplateResource> libraryResources;
 
   /**
    * @param uri URI of the template library
    * @param root Element representing the Templates tag of this library
    */
-  public TemplateLibrary(Uri uri, Element root) throws GadgetException, TemplateParserException {
-    this(uri, root, false);
+  public XmlTemplateLibrary(Uri uri, Element root, String source) 
+      throws GadgetException, TemplateParserException {
+    this(uri, root, source, false);
   }
 
   /**
@@ -64,11 +67,21 @@
    * @param root Element representing the Templates tag of this library
    * @param safe Is this library exempt from being sanitized?
    */
-  public TemplateLibrary(Uri uri, Element root, boolean safe) 
+  public XmlTemplateLibrary(Uri uri, Element root, String source, boolean safe) 
       throws GadgetException, TemplateParserException {
-    libraryUri = uri;
-    registry = new DefaultTagRegistry(parseLibraryDocument(root));
+    this.libraryUri = uri;
+    this.source = source;
+    this.registry = new DefaultTagRegistry(parseLibraryDocument(root));
     this.safe = safe;
+    ImmutableSet.Builder<TemplateResource> resources = ImmutableSet.builder();
+    if (style != null) {
+      resources.add(TemplateResource.newStyleResource(style, this));
+    }
+    if (javaScript != null) {
+      resources.add(TemplateResource.newJavascriptResource(javaScript, this));
+    }
+    
+    this.libraryResources = resources.build();
   }
 
   /**
@@ -87,13 +100,6 @@
   }
   
   /**
-   * @return the concatenated contents of Style elements for the library.
-   */
-  public String getStyle() {
-    return style;
-  }
-  
-  /**
    * @return this library is safe and its content doesn't need to be sanitized. 
    */
   public boolean isSafe() {
@@ -101,11 +107,10 @@
   }
   
   /**
-   * TODO: Minify javascript (here or elsewhere)
-   * @return the concatenated contents of JavaScript elements for the library.
+   * @return This library as XML source.
    */
-  public String getJavaScript() {
-    return javaScript;
+  public String serialize() {
+    return source;
   }
   
   private Set<TagHandler> parseLibraryDocument(Element root)
@@ -143,20 +148,23 @@
       throw new TemplateParserException("Missing tag attribute on TemplateDef");
     }
 
-    Element templateElement = (Element) DomUtil.getFirstNamedChildNode(defElement, TEMPLATE_TAG);
-    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement);
-    if (handler != null) {
-      handlers.add(handler);
-    }
+    ImmutableSet.Builder<TemplateResource> resources = ImmutableSet.builder();
     
     Element scriptElement = (Element) DomUtil.getFirstNamedChildNode(defElement, JAVASCRIPT_TAG);
     if (scriptElement != null) {
-      processJavaScript(scriptElement);
+      resources.add(TemplateResource.newJavascriptResource(scriptElement.getTextContent(), this));
     }
     
     Element styleElement = (Element) DomUtil.getFirstNamedChildNode(defElement, STYLE_TAG);
     if (styleElement != null) {
-      processStyle(styleElement);
+      resources.add(TemplateResource.newStyleResource(styleElement.getTextContent(), this));
+    }
+
+    Element templateElement = (Element) DomUtil.getFirstNamedChildNode(defElement, TEMPLATE_TAG);
+    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement,
+        resources.build());
+    if (handler != null) {
+      handlers.add(handler);
     }
   }
 
@@ -167,7 +175,8 @@
       throw new TemplateParserException("Missing tag attribute on Template");
     }
     
-    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement);
+    TagHandler handler = createHandler(tagAttribute.getNodeValue(), templateElement,
+        ImmutableSet.<TemplateResource>of());
     if (handler != null) {
       handlers.add(handler);
     }
@@ -205,7 +214,8 @@
     }
   }
 
-  private TagHandler createHandler(String tagName, Element template)
+  private TagHandler createHandler(String tagName, Element template,
+      Set<TemplateResource> resources)
       throws TemplateParserException {
     String [] nameParts = tagName.split(":");
     // At this time, we only support namespaced tags
@@ -217,6 +227,47 @@
       throw new TemplateParserException(
           "Can't create tags in undeclared namespace: " + nameParts[0]);
     }
-    return new TemplateBasedTagHandler(template, namespaceUri, nameParts[1]);
+    
+    return new LibraryTagHandler(
+        new TemplateBasedTagHandler(template, namespaceUri, nameParts[1]),
+        resources);
+  }
+  
+  /**
+   * TagHandler delegate reponsible for adding necessary tag resources
+   * as each tag gets processed. 
+   */
+  private class LibraryTagHandler implements TagHandler {
+    private final TagHandler tagHandler;
+    private final Set<TemplateResource> tagResources;
+
+    public LibraryTagHandler(TagHandler tagHandler, Set<TemplateResource> resources) {
+      this.tagHandler = tagHandler;
+      tagResources = resources;
+    }
+
+    public String getNamespaceUri() {
+      return tagHandler.getNamespaceUri();
+    }
+
+    public String getTagName() {
+      return tagHandler.getTagName();
+    }
+
+    public void process(Node result, Element tag, TemplateProcessor processor) {
+      // Add all template resources and library resources.  Use the resource
+      // instance as its own key, since we're careful to create the resource
+      // objects once.  NOTE: this assumes that TemplateResource uses instance
+      // equality, not value equality.
+      for (TemplateResource resource : tagResources) {
+        processor.getTemplateContext().addResource(resource, resource);
+      }
+      
+      for (TemplateResource resource : libraryResources) {
+        processor.getTemplateContext().addResource(resource, resource);
+      }
+
+      tagHandler.process(result, tag, processor);
+    }
   }
 }

Propchange: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/templates/XmlTemplateLibrary.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Modified: incubator/shindig/trunk/java/gadgets/src/main/resources/org/apache/shindig/gadgets/templates/OSML_library.xml
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/resources/org/apache/shindig/gadgets/templates/OSML_library.xml?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/resources/org/apache/shindig/gadgets/templates/OSML_library.xml (original)
+++ incubator/shindig/trunk/java/gadgets/src/main/resources/org/apache/shindig/gadgets/templates/OSML_library.xml Fri May 29 21:15:53 2009
@@ -1,3 +1,22 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
 <Templates xmlns:os="http://ns.opensocial.org/2008/markup">
   <Namespace prefix="os" url="http://ns.opensocial.org/2008/markup"/>
   <Template tag="os:Name">

Modified: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/TemplateRewriterTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/TemplateRewriterTest.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/TemplateRewriterTest.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/TemplateRewriterTest.java Fri May 29 21:15:53 2009
@@ -24,6 +24,7 @@
 
 import org.apache.shindig.common.uri.Uri;
 import org.apache.shindig.common.xml.XmlUtil;
+import org.apache.shindig.config.AbstractContainerConfig;
 import org.apache.shindig.expressions.Expressions;
 import org.apache.shindig.gadgets.Gadget;
 import org.apache.shindig.gadgets.GadgetContext;
@@ -33,20 +34,26 @@
 import org.apache.shindig.gadgets.render.FakeMessageBundleFactory;
 import org.apache.shindig.gadgets.spec.GadgetSpec;
 import org.apache.shindig.gadgets.spec.SpecParserException;
+import org.apache.shindig.gadgets.templates.AbstractTagHandler;
 import org.apache.shindig.gadgets.templates.DefaultTagRegistry;
 import org.apache.shindig.gadgets.templates.DefaultTemplateProcessor;
 import org.apache.shindig.gadgets.templates.TagHandler;
 import org.apache.shindig.gadgets.templates.TemplateLibrary;
 import org.apache.shindig.gadgets.templates.TemplateLibraryFactory;
 import org.apache.shindig.gadgets.templates.TemplateProcessor;
+import org.apache.shindig.gadgets.templates.XmlTemplateLibrary;
 import org.json.JSONException;
 import org.json.JSONObject;
 import org.junit.Before;
 import org.junit.Test;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
 
+import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
 import com.google.inject.Provider;
 
 /** 
@@ -58,6 +65,7 @@
   private Gadget gadget;
   private MutableContent content;
   private TemplateRewriter rewriter;
+  private final Map<String, Object> config = Maps.newHashMap();
   
   private static final Uri GADGET_URI = Uri.parse("http://example.org/gadget.php");
   
@@ -77,7 +85,7 @@
     "<script type='text/os-template' name='myTemplate'>Hello, ${user.name}</script>";  
   
   private static final String CONTENT_WITH_TAG =
-    "<script type='text/os-template' xmlns:foo='#foo' tag='foo:Bar'>Hello, ${user.name}</script>";  
+    "<script type='text/os-template' xmlns:foo='#foo' tag='foo:Bar'>Hello, ${user.name}</script>";
   
   private static final String CONTENT_WITH_AUTO_UPDATE =
     "<script type='text/os-template' autoUpdate='true'>Hello, ${user.name}</script>";
@@ -87,16 +95,25 @@
     "  <Namespace prefix='my' url='#my'/>" +
     "  <JavaScript>script</JavaScript>" +
     "  <Style>style</Style>" +
-    "  <Template tag='my:Tag'>library tag</Template>" +
+    "  <Template tag='my:Tag1'>external1</Template>" +
+    "  <Template tag='my:Tag2'>external2</Template>" +
+    "  <Template tag='my:Tag3'>external3</Template>" +
+    "  <Template tag='my:Tag4'>external4</Template>" +
     "</Templates>";
 
   private static final String TEMPLATE_LIBRARY_URI = "http://example.org/library.xml";
   private static final String CONTENT_WITH_TAG_FROM_LIBRARY =
-    "<script type='text/os-template' xmlns:my='#my'><my:Tag/></script>";  ;
-  
+    "<script type='text/os-template' xmlns:my='#my'><my:Tag4/></script>";  ;
+ 
+  private static final String CONTENT_TESTING_PRECEDENCE_RULES =
+    "<script type='text/os-template' xmlns:my='#my' tag='my:Tag1'>inline1</script>" +
+    "<script type='text/os-template' xmlns:my='#my' tag='my:Tag2'>inline2</script>" +
+    "<script type='text/os-template' xmlns:my='#my' tag='my:Tag3'>inline3</script>" +
+    "<script type='text/os-template' xmlns:my='#my'><my:Tag1/><my:Tag2/><my:Tag3/><my:Tag4/></script>";  ;
+
   @Before
   public void setUp() {
-    Set<TagHandler> handlers = ImmutableSet.of();
+    Set<TagHandler> handlers = ImmutableSet.of(testTagHandler("Tag1", "default1"));
     rewriter = new TemplateRewriter(
         new Provider<TemplateProcessor>() {
           public TemplateProcessor get() {
@@ -106,9 +123,18 @@
         new FakeMessageBundleFactory(),
         new Expressions(),
         new DefaultTagRegistry(handlers), 
-        new FakeTemplateLibraryFactory());
+        new FakeTemplateLibraryFactory(),
+        new FakeContainerConfig());
   }
   
+ private static TagHandler testTagHandler(String name, final String content) {
+   return new AbstractTagHandler("#my", name) {
+    public void process(Node result, Element tag, TemplateProcessor processor) {
+      result.appendChild(result.getOwnerDocument().createTextNode(content));
+    }
+   };
+ }
+ 
   @Test
   public void simpleTemplate() throws Exception {
     // Render a simple template
@@ -178,8 +204,46 @@
     assertTrue("Style not inserted", content.getContent().indexOf(
         "<style type=\"text/css\">style</style>") > 0);
     assertTrue("Tag not executed", content.getContent().indexOf(
-        "library tag") > 0);
+        "external4") > 0);
+    
+    testFeatureRemoved();
+  }
+  
+  @Test
+  public void tagPrecedenceRules() throws Exception {
+    // Tag definitions include:
+    // Default handlers: tag1 default1
+    // OSML: tag1 osml1 tag2 osml2
+    // inline tags: tag1 inline1 tag2 inline2 tag3 inline3
+    // External tags: tag1 external1 tag2 external2 tag3 external3 tag4 external4
     
+    config.put("${Cur['gadgets.features'].osml.library}",
+        "org/apache/shindig/gadgets/rewrite/OSML_test.xml");
+
+    setupGadget(getGadgetXmlWithLibrary(CONTENT_TESTING_PRECEDENCE_RULES));
+    rewriter.rewrite(gadget, content);
+    assertTrue("Precedence rules violated",
+        content.getContent().indexOf("default1osml2inline3external4") > 0);
+   
+    testFeatureRemoved();
+  }
+  
+  @Test
+  public void tagPrecedenceRulesWithoutOSML() throws Exception {
+    // Tag definitions include:
+    // Default handlers: tag1 default1
+    // OSML: tag1 osml1 tag2 osml2
+    // inline tags: tag1 inline1 tag2 inline2 tag3 inline3
+    // External tags: tag1 external1 tag2 external2 tag3 external3 tag4 external4
+
+    // Explicitly don't support OSML
+    config.put("${Cur['gadgets.features'].osml.library}", "");
+
+    setupGadget(getGadgetXmlWithLibrary(CONTENT_TESTING_PRECEDENCE_RULES));
+    rewriter.rewrite(gadget, content);
+    assertTrue("Precedence rules violated",
+        content.getContent().indexOf("default1inline2inline3external4") > 0);
+   
     testFeatureRemoved();
   }
   
@@ -273,7 +337,16 @@
     public TemplateLibrary loadTemplateLibrary(GadgetContext context, Uri uri)
         throws GadgetException {
       assertEquals(TEMPLATE_LIBRARY_URI, uri.toString());
-      return new TemplateLibrary(uri, XmlUtil.parseSilent(TEMPLATE_LIBRARY));
+      return new XmlTemplateLibrary(uri, XmlUtil.parseSilent(TEMPLATE_LIBRARY), 
+          TEMPLATE_LIBRARY);
+    }
+  }
+  
+  private class FakeContainerConfig extends AbstractContainerConfig {
+    @Override
+    public Object getProperty(String container, String name) {
+      return config.get(name);
     }
+    
   }
 }

Modified: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/DefaultTemplateProcessorTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/DefaultTemplateProcessorTest.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/DefaultTemplateProcessorTest.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/DefaultTemplateProcessorTest.java Fri May 29 21:15:53 2009
@@ -228,7 +228,7 @@
     // This is a little hacky but is fine for testing purposes. Assumes that DOM implementation
     // is based on Xerces which will always has a userData hashtable
     Document doc = singletonElementHandler.elem.getOwnerDocument();
-    Class docClass = doc.getClass();
+    Class<?> docClass = doc.getClass();
     Field userDataField = null;
     while (userDataField == null) {
       try {
@@ -240,7 +240,7 @@
     }
     // Access is typically protected so just bypass
     userDataField.setAccessible(true);
-    Hashtable userDataMap = (Hashtable)userDataField.get(doc);
+    Hashtable<?, ?> userDataMap = (Hashtable<?, ?>) userDataField.get(doc);
 
     // There should be only one element in the user data map, if there are more then the
     // cloning process has put them there which can be a nasty source of memory leaks. Consider

Modified: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/RenderTagHandlerTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/RenderTagHandlerTest.java?rev=780108&r1=780107&r2=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/RenderTagHandlerTest.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/RenderTagHandlerTest.java Fri May 29 21:15:53 2009
@@ -37,6 +37,7 @@
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Lists;
@@ -99,12 +100,13 @@
     Element tagInstance = parseTemplate(tagMarkup);
     
     templateDef.getOwnerDocument().adoptNode(tagInstance);
-    TemplateBasedTagHandler tagHandler = 
+    TagHandler tagHandler = 
       new TemplateBasedTagHandler(tagInstance, TEST_NS, tagName);
 
-    TagRegistry reg = new CompositeTagRegistry(Lists.newArrayList(
-        registry, new DefaultTagRegistry(ImmutableSet.of((TagHandler)tagHandler))));
-    
+    TagRegistry reg = new CompositeTagRegistry(ImmutableList.of(
+        registry,
+        new DefaultTagRegistry(ImmutableSet.of(tagHandler))));
+        
     DocumentFragment result = processor.processTemplate(templateDef, context, resolver, reg);
     String output = serialize(result);
     assertEquals(expectedResult, output);

Copied: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java (from r779276, incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/TemplateLibraryTest.java)
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java?p2=incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java&p1=incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/TemplateLibraryTest.java&r1=779276&r2=780108&rev=780108&view=diff
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/TemplateLibraryTest.java (original)
+++ incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java Fri May 29 21:15:53 2009
@@ -24,38 +24,44 @@
 
 import org.apache.shindig.common.uri.Uri;
 import org.apache.shindig.common.xml.XmlUtil;
+import org.apache.shindig.expressions.Expressions;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+
+import com.google.common.collect.ImmutableMap;
 
 /**
  * Test for TemplateLibrary parsing.
  * 
  * TODO: Parse failure tests
  */
-public class TemplateLibraryTest {
+public class XmlTemplateLibraryTest {
 
   public static final String LIB_MARKUP = 
     "<Templates xmlns:my='#my'>" +
     "  <Namespace prefix='my' url='#my'/>" +
-    "  <JavaScript>script</JavaScript>" +
-    "  <JavaScript>script2</JavaScript>" +
-    "  <Style>style</Style>" +
-    "  <Style>style2</Style>" +
+    "  <JavaScript>libscript</JavaScript>" +
+    "  <JavaScript>libscript2</JavaScript>" +
+    "  <Style>libstyle</Style>" +
+    "  <Style>libstyle2</Style>" +
     "  <Template tag='my:Flat'>Flat tag</Template>" +
     "  <TemplateDef tag='my:Def'>" +
     "    <Template>Def tag</Template>" +
-    "    <JavaScript>script3</JavaScript>" +
-    "    <Style>style3</Style>" +
+    "    <JavaScript>tagscript</JavaScript>" +
+    "    <Style>tagstyle</Style>" +
     "  </TemplateDef>" +
     "</Templates>";
   
   private static TemplateLibrary lib;
+
+  private static Element doc;
   
   @BeforeClass
   public static void createDefaultLibrary() throws Exception {
-    Element doc = XmlUtil.parse(LIB_MARKUP);
-    lib = new TemplateLibrary(Uri.parse("http://example.com/my"), doc);
+    doc = XmlUtil.parse(LIB_MARKUP);
+    lib = new XmlTemplateLibrary(Uri.parse("http://example.com/my"), doc, LIB_MARKUP);
   }
   
   @Test
@@ -78,12 +84,58 @@
   }
 
   @Test
-  public void testScript() {
-    assertEquals("script\nscript2\nscript3", lib.getJavaScript());
+  public void testAddedResources() {
+    final TemplateContext context = new TemplateContext(null, ImmutableMap.<String, Object>of());
+    TemplateProcessor processor = new DefaultTemplateProcessor(new Expressions()) {
+      @Override
+      public TemplateContext getTemplateContext() {
+        return context;
+      }      
+    };
+    
+    TagHandler handlerWithResources = lib.getTagRegistry()
+       .getHandlerFor(new TagRegistry.NSName("#my", "Def"));
+    TagHandler handlerWithNoResources = lib.getTagRegistry()
+        .getHandlerFor(new TagRegistry.NSName("#my", "Flat"));
+    
+    Node result = doc.getOwnerDocument().createDocumentFragment();
+    Element tag = doc.getOwnerDocument().createElement("test");
+    
+    // Script and style elements for the library should get registered
+    // with the first tag for the whole library
+    handlerWithNoResources.process(result, tag, processor);
+    assertEquals("<STYLE>libstyle\nlibstyle2</STYLE>" +
+    		"<JAVASCRIPT>libscript\nlibscript2</JAVASCRIPT>",
+    		serializeResources(context));
+
+    // Now script and style elements for the tag should get registered
+    handlerWithResources.process(result, tag, processor);
+    assertEquals("<STYLE>libstyle\nlibstyle2</STYLE>" +
+        "<JAVASCRIPT>libscript\nlibscript2</JAVASCRIPT>" +
+        "<JAVASCRIPT>tagscript</JAVASCRIPT>" +
+        "<STYLE>tagstyle</STYLE>",
+        serializeResources(context));
+
+    // Nothing new should get registered with one more call
+    handlerWithResources.process(result, tag, processor);
+    assertEquals("<STYLE>libstyle\nlibstyle2</STYLE>" +
+        "<JAVASCRIPT>libscript\nlibscript2</JAVASCRIPT>" +
+        "<JAVASCRIPT>tagscript</JAVASCRIPT>" +
+        "<STYLE>tagstyle</STYLE>",
+        serializeResources(context));
   }
   
+  private String serializeResources(TemplateContext context) {
+    StringBuilder builder = new StringBuilder();
+    for (TemplateResource resource : context.getResources()) {
+      builder.append(resource);
+    }
+    
+    return builder.toString();
+  }
   @Test
-  public void testStyle() {
-    assertEquals("style\nstyle2\nstyle3", lib.getStyle());
+  public void testSerialize() {
+    assertEquals(LIB_MARKUP, lib.serialize());
   }
+  
 }

Propchange: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/templates/XmlTemplateLibraryTest.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Added: incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml?rev=780108&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml (added)
+++ incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml Fri May 29 21:15:53 2009
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-->
+<!-- 
+  An "OSML" library used to test precedence rules.  The OSML library
+  may not override built-in tags, but cannot be overridden by
+  user-defined tags.   Used by TemplateRewriterTest.
+-->
+<Templates xmlns:my="#my">
+  <Namespace prefix="my" url="#my"/>
+  <Template tag='my:Tag1'>osml1</Template>
+  <Template tag='my:Tag2'>osml2</Template>
+</Templates>

Propchange: incubator/shindig/trunk/java/gadgets/src/test/resources/org/apache/shindig/gadgets/rewrite/OSML_test.xml
------------------------------------------------------------------------------
    svn:eol-style = native