You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@shindig.apache.org by jo...@apache.org on 2008/08/21 23:48:56 UTC

svn commit: r687871 - in /incubator/shindig/trunk/java/gadgets/src: main/java/org/apache/shindig/gadgets/rewrite/ test/java/org/apache/shindig/gadgets/rewrite/

Author: johnh
Date: Thu Aug 21 14:48:55 2008
New Revision: 687871

URL: http://svn.apache.org/viewvc?rev=687871&view=rev
Log:
Rewrite of DefaultContentRewriter to use parse tree-based rewriting APIs rather than the lexer.

This is the first step toward migrating away from lexer-based rewriting and toward a modular tree-based rewriter.

Next steps:
* Compare end-to-end behavior of two rewriters.
* Compare performance numbers between two approaches.
* Shift out DefaultContentRewriter in favor of ParseTreeDefaultContentRewriter, when functionality/perf proves good.
* Modularize functionality (split ParseTreeHtmlRewriter into separate, focused classes).


Added:
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeDefaultContentRewriter.java
    incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriter.java
    incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriterTest.java

Added: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeDefaultContentRewriter.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeDefaultContentRewriter.java?rev=687871&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeDefaultContentRewriter.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeDefaultContentRewriter.java Thu Aug 21 14:48:55 2008
@@ -0,0 +1,178 @@
+/*
+ * 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.rewrite;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import com.google.inject.name.Named;
+
+import org.apache.shindig.gadgets.GadgetException;
+import org.apache.shindig.gadgets.GadgetSpecFactory;
+import org.apache.shindig.gadgets.http.HttpRequest;
+import org.apache.shindig.gadgets.http.HttpResponse;
+import org.apache.shindig.gadgets.parse.GadgetHtmlParser;
+import org.apache.shindig.gadgets.spec.GadgetSpec;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStreamWriter;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.net.URI;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Default implementation of content rewriting.
+ */
+@Singleton
+public class ParseTreeDefaultContentRewriter implements ContentRewriter {
+
+  private final GadgetSpecFactory specFactory;
+  private final GadgetHtmlParser htmlParser;
+  private final String includeUrls;
+  private final String excludeUrls;
+  private final String expires;
+  private final Set<String> includeTags;
+  private final ParseTreeHtmlRewriter htmlRewriter;
+
+  @Inject
+  public ParseTreeDefaultContentRewriter(
+      GadgetSpecFactory specFactory,
+      GadgetHtmlParser htmlParser,
+      @Named("shindig.content-rewrite.include-urls")String includeUrls,
+      @Named("shindig.content-rewrite.exclude-urls")String excludeUrls,
+      @Named("shindig.content-rewrite.expires")String expires,
+      @Named("shindig.content-rewrite.include-tags")String includeTags) {
+    this.specFactory = specFactory;
+    this.htmlParser = htmlParser;
+    this.includeUrls = includeUrls;
+    this.excludeUrls = excludeUrls;
+    this.expires = expires;
+    this.includeTags = new HashSet<String>();
+    for (String s : includeTags.split(",")) {
+      if (s != null && s.trim().length() > 0) {
+        this.includeTags.add(s.trim().toLowerCase());
+      }
+    }
+    htmlRewriter = new ParseTreeHtmlRewriter(this.htmlParser);
+  }
+
+  public HttpResponse rewrite(HttpRequest request, HttpResponse original) {
+    try {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream(
+          (original.getContentLength() * 110) / 100);
+      OutputStreamWriter output = new OutputStreamWriter(baos,
+          original.getEncoding());
+      String mimeType = original.getHeader("Content-Type");
+      if (request.getRewriteMimeType() != null) {
+        mimeType = request.getRewriteMimeType();
+      }
+      GadgetSpec spec = null;
+      if (request.getGadget() != null) {
+        spec = specFactory.getGadgetSpec(request.getGadget().toJavaUri(), false);
+      }
+      if (rewrite(spec, request.getUri().toJavaUri(),
+          original.getResponseAsString(),
+          mimeType,
+          output)) {
+        return new HttpResponse(original.getHttpStatusCode(),
+            baos.toByteArray(),
+            original.getAllHeaders(),
+            original.getEncoding());
+      }
+      return null;
+    } catch (UnsupportedEncodingException uee) {
+      throw new RuntimeException(uee);
+    } catch (GadgetException ge) {
+      return null;
+    }
+  }
+
+  public String rewriteGadgetView(GadgetSpec spec, String view, String mimeType) {
+    StringWriter sw = new StringWriter();
+    if (rewrite(spec, spec.getUrl(), view, mimeType, sw)) {
+      return sw.toString();
+    } else {
+      return null;
+    }
+  }
+
+  private boolean rewrite(GadgetSpec spec, URI source, String s, String mimeType, Writer w) {
+    // Dont rewrite content if the spec is unavailable
+    if (spec == null) {
+      return false;
+    }
+
+    // Store the feature in the spec so we dont keep parsing it
+    ContentRewriterFeature rewriterFeature = (ContentRewriterFeature) spec
+        .getAttribute("content-rewrite");
+    if (rewriterFeature == null) {
+      rewriterFeature = new ContentRewriterFeature(spec, includeUrls, excludeUrls, expires,
+          includeTags);
+      spec.setAttribute("content-rewrite", rewriterFeature);
+    }
+
+    if (!rewriterFeature.isRewriteEnabled()) {
+      return false;
+    }
+    
+    if (isHTML(mimeType)) {
+      LinkRewriter linkRewriter = createLinkRewriter(spec, rewriterFeature);
+      htmlRewriter.rewrite(s, source, spec, rewriterFeature, linkRewriter, w);
+      return true;
+    } else if (isCSS(mimeType)) {
+      if (getProxyUrl() != null) {
+        CssRewriter.rewrite(new StringReader(s), source,
+            createLinkRewriter(spec, rewriterFeature), w);
+        return true;
+      } else {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  private boolean isHTML(String mime) {
+    if (mime == null) {
+      return false;
+    }
+    return (mime.toLowerCase().contains("html"));
+  }
+
+  private boolean isCSS(String mime) {
+    if (mime == null) {
+      return false;
+    }
+    return (mime.toLowerCase().contains("css"));
+  }
+
+  protected String getProxyUrl() {
+    return "/gadgets/proxy?url=";
+  }
+
+  protected String getConcatUrl() {
+    return "/gadgets/concat?";
+  }
+
+  protected LinkRewriter createLinkRewriter(GadgetSpec spec,
+      ContentRewriterFeature rewriterFeature) {
+    return new ProxyingLinkRewriter(spec.getUrl(), rewriterFeature, getProxyUrl());
+  }
+}

Added: incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriter.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriter.java?rev=687871&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriter.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/main/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriter.java Thu Aug 21 14:48:55 2008
@@ -0,0 +1,363 @@
+/*
+ * 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.rewrite;
+
+import com.google.caja.lexer.HtmlTokenType;
+import com.google.caja.lexer.Token;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.shindig.common.util.Utf8UrlCoder;
+import org.apache.shindig.gadgets.GadgetException;
+import org.apache.shindig.gadgets.parse.GadgetHtmlNode;
+import org.apache.shindig.gadgets.parse.GadgetHtmlParser;
+import org.apache.shindig.gadgets.parse.ParsedHtmlNode;
+import org.apache.shindig.gadgets.servlet.ProxyBase;
+import org.apache.shindig.gadgets.spec.GadgetSpec;
+
+/**
+ * Rewrites HTML, subsuming a significant amount of rewriting
+ * functionality previously broken out as HtmlTagTransformers.
+ * Uses parse tree rather than lexer to achieve its ends.
+ */
+public class ParseTreeHtmlRewriter {
+  
+  private GadgetHtmlParser htmlParser;
+
+  public ParseTreeHtmlRewriter(GadgetHtmlParser htmlParser) {
+    this.htmlParser = htmlParser;
+  }
+
+  public void rewrite(String content, URI source,
+      GadgetSpec spec, ContentRewriterFeature rewriterFeature,
+      LinkRewriter linkRewriter, Writer writer) {
+    List<GadgetHtmlNode> nodes = getParsedHtmlNodes(content);
+    if (nodes == null) {
+      try {
+        // Can't rewrite something that's malformed:
+        // leave it alone.
+        writer.append(content);
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      return;
+    }
+    
+    // Temporary implementation: in-class three-pass rewriting,
+    // with each method performing the equivalent of one
+    // post-refactored ContentRewriter
+    transformLinkingTags(nodes, source, linkRewriter);
+    
+    if (getConcatUrl() != null &&
+        rewriterFeature != null &&
+        rewriterFeature.getIncludedTags().contains("script")) {
+      transformConsolidateJSTags(nodes, source, spec, rewriterFeature);
+    }
+    
+    if (rewriterFeature != null &&
+        rewriterFeature.getIncludedTags().contains("style")) {
+      transformStyleTags(nodes, source, linkRewriter);
+    }
+    
+    try {
+      for (GadgetHtmlNode node : nodes) {
+        node.render(writer);
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+  
+  // Formerly LinkingTagRewriter
+  private static final Map<String, Set<String>> tagAttributeTargets =
+      getDefaultTargets();
+
+  void transformLinkingTags(List<GadgetHtmlNode> nodes, URI gadgetUri,
+       LinkRewriter linkRewriter) {
+    Queue<GadgetHtmlNode> nodesToProcess =
+      new LinkedList<GadgetHtmlNode>();
+    nodesToProcess.addAll(nodes);
+  
+    while (!nodesToProcess.isEmpty()) {
+      GadgetHtmlNode curNode = nodesToProcess.remove();
+      if (!curNode.isText()) {
+        // Depth-first iteration over children. Order doesn't matter anyway.
+        nodesToProcess.addAll(curNode.getChildren());
+        
+        Set<String> curTagAttrs =
+            tagAttributeTargets.get(curNode.getTagName().toLowerCase());
+        if (curTagAttrs != null) {
+          for (String attrKey : curNode.getAttributeKeys()) {
+            if (curTagAttrs.contains(attrKey.toLowerCase())) {
+              String attrValue = curNode.getAttributeValue(attrKey);
+            
+              // Attribute marked for rewriting: do it!
+              curNode.setAttribute(attrKey,
+                  linkRewriter.rewrite(attrValue, gadgetUri));
+            }
+          }
+        }
+      }
+    }
+  }
+  
+  private static Map<String, Set<String>> getDefaultTargets() {
+    Map<String, Set<String>> targets  = new HashMap<String, Set<String>>();
+    targets.put("img", new HashSet<String>(Arrays.asList("src")));
+    targets.put("embed", new HashSet<String>(Arrays.asList("src")));
+    targets.put("link", new HashSet<String>(Arrays.asList("href")));
+    return targets;
+  }
+  
+  // Formerly JavascriptTagMerger
+  private final static int MAX_URL_LENGTH = 1500;
+
+  void transformConsolidateJSTags(List<GadgetHtmlNode> nodes,
+      URI gadgetUri, GadgetSpec spec, ContentRewriterFeature rewriterFeature) {
+    // Add top-level node and remove it afterward to simplify
+    // algorithm to: pop node, process children, continue
+    GadgetHtmlNode wrapper = new GadgetHtmlNode("wrapper", null);
+    for (GadgetHtmlNode node : nodes) {
+      wrapper.appendChild(node);
+    }
+    doJSTagConsolidation(getJsConcatBase(spec, rewriterFeature), wrapper, gadgetUri);
+    
+    // Replace nodes list with wrapper children.
+    // This stuff will be cleaned up with rewriter modularization by introducing
+    // a well-defined top-level node for processing purposes.
+    nodes.clear();
+    for (GadgetHtmlNode node : wrapper.getChildren()) {
+      // Dissociate from parent.
+      wrapper.removeChild(node);
+      nodes.add(node);
+    }
+  }
+  
+  String getJsConcatBase(GadgetSpec spec, ContentRewriterFeature rewriterFeature) {
+    return getConcatUrl() +
+           ProxyBase.REWRITE_MIME_TYPE_PARAM +
+           "=text/javascript&" +
+           "gadget=" +
+           Utf8UrlCoder.encode(spec.getUrl().toString()) +
+           "&fp=" +
+           rewriterFeature.getFingerprint() +
+           '&';
+  }
+  
+  // TODO: add type="text/javascript" to script tag, with tests
+  // (requires guaranteed attribute ordering)
+  private void doJSTagConsolidation(String concatBase,
+      GadgetHtmlNode sourceNode, URI baseUri) {
+    // Bootstrap queue of children over which to iterate,
+    // ie. lists of siblings to potentially combine
+    Queue<GadgetHtmlNode> nodesToProcess =
+        new LinkedList<GadgetHtmlNode>();
+    nodesToProcess.add(sourceNode);
+    
+    while (!nodesToProcess.isEmpty()) {
+      GadgetHtmlNode parentNode = nodesToProcess.remove();
+      if (!parentNode.isText()) {
+        List<GadgetHtmlNode> childList = parentNode.getChildren();
+        
+        // Iterate over children next in depth-first fashion.
+        // Text nodes (such as <script src> processed here) will be ignored.
+        nodesToProcess.addAll(childList);
+        
+        List<GadgetHtmlNode> toRemove = new ArrayList<GadgetHtmlNode>();
+        List<URI> scripts = new ArrayList<URI>();
+        boolean processScripts = false;
+        for (int i = 0; i < childList.size(); ++i) {
+          GadgetHtmlNode cur = childList.get(i);
+        
+          // Find consecutive <script src=...> tags
+          if (!cur.isText() &&
+               cur.getTagName().equalsIgnoreCase("script") &&
+               cur.hasAttribute("src")) {
+            URI scriptUri = null;
+            try {
+              scriptUri =
+                  baseUri.resolve(new URI(cur.getAttributeValue("src")));
+            } catch (URISyntaxException use) {
+              // Same behavior as JavascriptTagMerger
+              // Perhaps switch to ignoring script src instead?
+              throw new RuntimeException(use);
+            }
+            scripts.add(scriptUri);
+            toRemove.add(cur);
+          } else if (cur.isText() && cur.getText().matches("\\s*")) {
+            toRemove.add(cur);
+          } else if (scripts.size() > 0) {
+            processScripts = true;
+          }
+          
+          if (i == (childList.size() - 1)) {
+            processScripts = true;
+          }
+        
+          if (processScripts && scripts.size() > 0) {
+            // Tags found. Concatenate scripts together.
+            List<URI> concatUris = getConcatenatedUris(concatBase, scripts);
+            
+            // Insert concatenated nodes before first match
+            for (URI concatUri : concatUris) {
+              GadgetHtmlNode newScript = new GadgetHtmlNode("script", null);
+              newScript.setAttribute("src", concatUri.toString());
+              parentNode.insertBefore(newScript, toRemove.get(0));
+            }
+            
+            // Remove contributing match nodes
+            for (GadgetHtmlNode remove : toRemove) {
+              parentNode.removeChild(remove);
+            }
+            
+            processScripts = false;
+            scripts.clear();
+            toRemove.clear();
+          }
+        }
+      }
+    }
+  }
+  
+  private List<URI> getConcatenatedUris(String concatBase, List<URI> uris) {
+    List<URI> concatUris = new LinkedList<URI>();
+    int paramIndex = 1;
+    StringBuilder builder = null;
+    int maxUriLen = MAX_URL_LENGTH + concatBase.length();
+    try {
+      int uriIx = 0, lastUriIx = (uris.size() - 1);
+      for (URI uri : uris) {
+        if (paramIndex == 1) {
+          builder = new StringBuilder(concatBase);
+        } else {
+          builder.append("&");
+        }
+        builder.append(paramIndex).append("=")
+            .append(URLEncoder.encode(uri.toString(), "UTF-8"));
+        if (builder.length() > maxUriLen ||
+            uriIx == lastUriIx) {
+          // Went over URI length warning limit or on the last uri
+          concatUris.add(new URI(builder.toString()));
+          builder = null;
+          paramIndex = 0;
+        }
+        ++paramIndex;
+        ++uriIx;
+      }
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException(e);
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e);
+    }
+    return concatUris;
+  }
+  
+  protected String getConcatUrl() {
+    return "/gadgets/concat?";
+  }
+  
+  // Formerly StyleTagRewriter
+  void transformStyleTags(List<GadgetHtmlNode> nodes, URI gadgetUri,
+      LinkRewriter linkRewriter) {
+    Queue<GadgetHtmlNode> nodesToProcess =
+      new LinkedList<GadgetHtmlNode>();
+    nodesToProcess.addAll(nodes);
+  
+    while (!nodesToProcess.isEmpty()) {
+      GadgetHtmlNode curNode = nodesToProcess.remove();
+      if (!curNode.isText()) {
+        // Depth-first iteration over children. Order doesn't matter anyway.
+        nodesToProcess.addAll(curNode.getChildren());
+        
+        if (curNode.getTagName().equalsIgnoreCase("style")) {
+          String styleText = getNodeChildText(curNode);
+          curNode.clearChildren();
+          curNode.appendChild(new GadgetHtmlNode(
+              CssRewriter.rewrite(styleText, gadgetUri, linkRewriter)));
+        }
+      }
+    }
+  }
+  
+  private static String getNodeChildText(GadgetHtmlNode node) {
+    // TODO: move this to GadgetHtmlNode as a helper
+    StringBuilder builder = new StringBuilder();
+    for (GadgetHtmlNode child : node.getChildren()) {
+      if (child.isText()) {
+        builder.append(child.getText());
+      }
+    }
+    return builder.toString();
+  }
+  
+  private List<GadgetHtmlNode> getParsedHtmlNodes(String source) {
+    List<ParsedHtmlNode> parsed = null;
+    try {
+      parsed = htmlParser.parse(source);
+    } catch (GadgetException e) {
+      // Can't rewrite something that can't be parsed
+      return null;
+    }
+    
+    if (parsed == null) {
+      return null;
+    }
+    
+    List<GadgetHtmlNode> nodes = new LinkedList<GadgetHtmlNode>();
+    for (ParsedHtmlNode parsedNode : parsed) {
+      nodes.add(new GadgetHtmlNode(parsedNode));
+    }
+    return nodes;
+  }
+
+
+  public static String producePreTokenSeparator(Token<HtmlTokenType> token,
+      Token<HtmlTokenType> lastToken) {
+    if (token.type == HtmlTokenType.ATTRNAME) {
+      return " ";
+    }
+    if (token.type == HtmlTokenType.ATTRVALUE &&
+        lastToken != null &&
+        lastToken.type == HtmlTokenType.ATTRNAME) {
+      return "=";
+    }
+    return "";
+  }
+
+
+  public static String producePostTokenSeparator(Token<HtmlTokenType> token,
+      Token<HtmlTokenType> lastToken) {
+    return "";
+  }
+
+}

Added: incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriterTest.java
URL: http://svn.apache.org/viewvc/incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriterTest.java?rev=687871&view=auto
==============================================================================
--- incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriterTest.java (added)
+++ incubator/shindig/trunk/java/gadgets/src/test/java/org/apache/shindig/gadgets/rewrite/ParseTreeHtmlRewriterTest.java Thu Aug 21 14:48:55 2008
@@ -0,0 +1,365 @@
+/*
+ * 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.rewrite;
+
+import static org.easymock.EasyMock.expect;
+import org.easymock.classextension.EasyMock;
+
+import org.apache.shindig.gadgets.parse.GadgetHtmlParser;
+import org.apache.shindig.gadgets.parse.GadgetHtmlNodeTest;
+import org.apache.shindig.gadgets.parse.ParsedHtmlNode;
+
+import org.apache.shindig.gadgets.rewrite.ContentRewriterFeature;
+import org.apache.shindig.gadgets.rewrite.LinkRewriter;
+import org.apache.shindig.gadgets.rewrite.ParseTreeHtmlRewriter;
+import org.apache.shindig.gadgets.spec.GadgetSpec;
+
+import java.io.StringWriter;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Test the HTML rewriter foundation for basic operation
+ */
+public class ParseTreeHtmlRewriterTest extends BaseRewriterTestCase {
+
+  private URI baseUri;
+  private LinkRewriter pfxLinkRewriter;
+  private LinkRewriter noOpLinkRewriter;
+  private ContentRewriterFeature jsFeature;
+  private ContentRewriterFeature styleFeature;
+  private GadgetSpec spec;
+  private String concatBase;
+  private static final String LINK_PREFIX = "px-";
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    baseUri = new URI("http://gadget.org/dir/gadget.xml");
+    pfxLinkRewriter = new LinkRewriter() {
+      public String rewrite(String uri, URI context) {
+        // Just prefixes with LINK_PREFIX
+        return LINK_PREFIX + uri;
+      }
+    };
+    noOpLinkRewriter = new LinkRewriter() {
+      public String rewrite(String uri, URI context) {
+        return uri;
+      }
+    };
+    jsFeature = makeFeature("script");
+    styleFeature = makeFeature("style");
+    spec = EasyMock.createNiceMock(GadgetSpec.class);
+    expect(spec.getUrl()).andReturn(baseUri).anyTimes();
+    org.easymock.classextension.EasyMock.replay(spec);
+    concatBase = new ParseTreeHtmlRewriter(null).getJsConcatBase(spec, jsFeature);
+  }
+  
+  private ContentRewriterFeature makeFeature(String... includedTags) {
+    ContentRewriterFeature rewriterFeature =
+        EasyMock.createNiceMock(ContentRewriterFeature.class);
+    Set<String> tags = new HashSet<String>();
+    for (String tag : includedTags) {
+      tags.add(tag);
+    }
+    expect(rewriterFeature.getIncludedTags()).andReturn(tags).anyTimes();
+    expect(rewriterFeature.getFingerprint()).andReturn(-840722081).anyTimes();
+    org.easymock.classextension.EasyMock.replay(rewriterFeature);
+    return rewriterFeature;
+  }
+
+  private String rewriteHelper(String s, ParsedHtmlNode[] p,
+      ContentRewriterFeature rf, LinkRewriter lw) throws Exception {
+    GadgetHtmlParser parser = EasyMock.createNiceMock(GadgetHtmlParser.class);
+    List<ParsedHtmlNode> expected = p != null ? Arrays.asList(p) : null;
+    expect(parser.parse(s)).andReturn(expected).anyTimes();
+    org.easymock.classextension.EasyMock.replay(parser);
+    ParseTreeHtmlRewriter hr = new ParseTreeHtmlRewriter(parser);
+    StringWriter sw = new StringWriter();
+    hr.rewrite(s, baseUri, spec, rf, lw, sw);
+    return sw.toString();
+  }
+  
+  public void testPreserveJunk() throws Exception {
+    String s = "<div id=notvalid name='horrorShow\" />\n"
+        + "</br>\n"
+        + "</div>";
+    // Unparseable, with no ContentRewriterFeature assertions
+    assertEquals(s, rewriteHelper(s, null, null, noOpLinkRewriter));
+  }
+
+  public void testPreserveScriptNoJSConcatNoLinkRewrite() throws Exception {
+    String s = "<script src=\"http://a.b.com/1.js\"></script>"
+        + "<script src=\"http://a.b.com/2.js\"></script>";
+    String[][] script1attr = { { "src", "http://a.b.com/1.js" } };
+    String[][] script2attr = { { "src", "http://a.b.com/2.js" } };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", script1attr, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", script2attr, null)
+    };
+    assertEquals(s, rewriteHelper(s, p, null, noOpLinkRewriter));
+  }
+
+  public void testPreserveCss() throws Exception {
+    String s = "<html><style>body { background-color:#7f7f7f }</style></html>";
+    ParsedHtmlNode[] styleKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("body { background-color:#7f7f7f }")
+    };
+    ParsedHtmlNode[] htmlKids = {
+      GadgetHtmlNodeTest.makeParsedTagNode("style", null, styleKids)
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("html", null, htmlKids)
+    };
+    assertEquals(s, rewriteHelper(s, p, null, noOpLinkRewriter));
+  }
+
+  public void testPreserveComment() throws Exception {
+    String s = "<script>  <!-- something here --></script>";
+    ParsedHtmlNode[] comment = {
+      GadgetHtmlNodeTest.makeParsedTextNode("  <!-- something here -->")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, comment)
+    };
+    assertEquals(s, rewriteHelper(s, p, null, noOpLinkRewriter));
+  }
+
+  public void testPreserveText() throws Exception {
+    String s = "<script>dontcare</script>";
+    ParsedHtmlNode[] text = {
+      GadgetHtmlNodeTest.makeParsedTextNode("dontcare")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, text)
+    };
+    assertEquals(s, rewriteHelper(s, p, null, noOpLinkRewriter));
+  }
+
+  // Eventaully we want the opposite.
+  public void testPreserveUselessWhitespace() throws Exception {
+    String s = "   <script>  \n</script>\n ";
+    ParsedHtmlNode[] scriptKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("  \n")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTextNode("   "),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids),
+      GadgetHtmlNodeTest.makeParsedTextNode("\n ")
+    };
+    assertEquals(s, rewriteHelper(s, p, null, noOpLinkRewriter));
+  }
+  
+  // Formerly from JavascriptTagMergerTest
+  // This will be refactored into its own Rewriter test once the Rewriter passes
+  // themselves are separated out.
+  public void testJSMergePreserveNoExternal() throws Exception {
+    String s = "<script>\n"
+        + "doSomething\n"
+        + "</script>";
+    ParsedHtmlNode[] scriptKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("\ndoSomething\n")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids)
+    };
+    assertEquals(s, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergePreserveNoScript() throws Exception {
+    String s
+        = "<html><div id=\"test\">ceci ne pas une script</div></html>";
+    String[][] attribs = { { "id", "test" } };
+    ParsedHtmlNode[] divKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("ceci ne pas une script")
+    };
+    ParsedHtmlNode[] htmlKids = {
+      GadgetHtmlNodeTest.makeParsedTagNode("div", attribs, divKids)
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("html", null, htmlKids)
+    };
+    assertEquals(s, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergePreserveWithComment() throws Exception {
+    String s = "<script>" +
+        "<!--\ndoSomething\n-->" +
+        "</script>";
+    ParsedHtmlNode[] scriptKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("<!--\ndoSomething\n-->")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids)
+    };
+    assertEquals(s, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeSingleScriptReWrite() throws Exception {
+    String s = "<script src=\"http://a.b.com/1.js\"></script>";
+    String[][] attribs = { { "src", "http://a.b.com/1.js" } };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attribs, null)
+    };
+    String rewritten
+        = "<script src=\"" + concatBase + "1=http%3A%2F%2Fa.b.com%2F1.js\"></script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeTwoScriptReWriteWithWhitespace() throws Exception {
+    String s = "<script src=\"http://a.b.com/1.js\"></script>\n"
+        + "<script src=\"http://a.b.com/2.js\"></script>";
+    String[][] attr1 = { { "src", "http://a.b.com/1.js" } };
+    String[][] attr2 = { { "src", "http://a.b.com/2.js" } };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr1, null),
+      GadgetHtmlNodeTest.makeParsedTextNode("\n"),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr2, null)
+    };
+    String rewritten
+        = "<script src=\"" + concatBase + "1=http%3A%2F%2Fa.b.com%2F1.js&2=http%3A%2F%2Fa.b.com%2F2.js\"></script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeLeadAndTrailingScriptReWrite() throws Exception {
+    String s = "<script>\n"
+        + "doSomething\n"
+        + "</script>"
+        + "<script src=\"http://a.b.com/1.js\"></script>"
+        + "<script src=\"http://a.b.com/2.js\"></script>"
+        + "<script>"
+        + "doSomething\n"
+        + "</script>";
+    String[][] attr1 = { { "src", "http://a.b.com/1.js" } };
+    String[][] attr2 = { { "src", "http://a.b.com/2.js" } };
+    ParsedHtmlNode[] scriptKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("\ndoSomething\n")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr1, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr2, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids)
+    };
+    String rewritten = "<script>\n"
+        + "doSomething\n"
+        + "</script>"
+        + "<script src=\"" + concatBase + "1=http%3A%2F%2Fa.b.com%2F1.js&2=http%3A%2F%2Fa.b.com%2F2.js\"></script>"
+        + "<script>\n"
+        + "doSomething\n"
+        + "</script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeInterspersed() throws Exception {
+    String s = "<script src=\"http://a.b.com/1.js\"></script>"
+        + "<script src=\"http://a.b.com/2.js\"></script>"
+        + "<script><!-- doSomething --></script>"
+        + "<script src=\"http://a.b.com/3.js\"></script>"
+        + "<script src=\"http://a.b.com/4.js\"></script>";
+    String[][] attr1 = { { "src", "http://a.b.com/1.js" } };
+    String[][] attr2 = { { "src", "http://a.b.com/2.js" } };
+    String[][] attr3 = { { "src", "http://a.b.com/3.js" } };
+    String[][] attr4 = { { "src", "http://a.b.com/4.js" } };
+    ParsedHtmlNode[] scriptKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode("<!-- doSomething -->")
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr1, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr2, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", null, scriptKids),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr3, null),
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr4, null)
+    };
+    String rewritten =
+        "<script src=\"" + concatBase + "1=http%3A%2F%2Fa.b.com%2F1.js&2=http%3A%2F%2Fa.b.com%2F2.js\"></script>" +
+        "<script><!-- doSomething --></script>" +
+        "<script src=\"" + concatBase + "1=http%3A%2F%2Fa.b.com%2F3.js&2=http%3A%2F%2Fa.b.com%2F4.js\"></script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeDerelativizeHostRelative() throws Exception {
+    String s = "<script src=\"/1.js\"></script>";
+    String[][] attr1 = { { "src", "/1.js" } };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr1, null)  
+    };
+    String rewritten
+        = "<script src=\"" + concatBase + "1=http%3A%2F%2Fgadget.org%2F1.js\"></script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+
+  public void testJSMergeDerelativizePathRelative() throws Exception {
+    String s = "<script src=\"1.js\"></script>";
+    String[][] attr1 = { { "src", "1.js" } };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("script", attr1, null)  
+    };
+    String rewritten
+        = "<script src=\"" + concatBase + "1=http%3A%2F%2Fgadget.org%2Fdir%2F1.js\"></script>";
+    assertEquals(rewritten, rewriteHelper(s, p, jsFeature, noOpLinkRewriter));
+  }
+  
+  // Formerly from LinkingTagRewriterTest
+  public void testLinkingTagStandardRewrite() throws Exception {
+    String s = "<img src=\"http://a.b.com/img.gif\"></img>\n"
+        + "<IMG src=\"http://a.b.com/img2.gif\"/>\n"
+        + "<eMbeD src=\"http://a.b.com/some.mov\"/>\n"
+        + "<link href=\"http://a.b.com/link.html\"></link>";
+    String[][] img1attrib = { { "src", "http://a.b.com/img.gif" } };
+    String[][] img2attrib = { { "src", "http://a.b.com/img2.gif" } };
+    String[][] emb1attrib = { { "src", "http://a.b.com/some.mov" } };
+    String[][] href1attr = { { "href", "http://a.b.com/link.html" } };
+    ParsedHtmlNode[] p = {
+        GadgetHtmlNodeTest.makeParsedTagNode("img", img1attrib, null),
+        GadgetHtmlNodeTest.makeParsedTextNode("\n"),
+        GadgetHtmlNodeTest.makeParsedTagNode("IMG", img2attrib, null),
+        GadgetHtmlNodeTest.makeParsedTextNode("\n"),
+        GadgetHtmlNodeTest.makeParsedTagNode("eMbeD", emb1attrib, null),
+        GadgetHtmlNodeTest.makeParsedTextNode("\n"),
+        GadgetHtmlNodeTest.makeParsedTagNode("link", href1attr, null)
+    };
+    String rewritten = "<img src=\"" + LINK_PREFIX + "http://a.b.com/img.gif\"/>\n"
+        + "<IMG src=\"" + LINK_PREFIX + "http://a.b.com/img2.gif\"/>\n"
+        + "<eMbeD src=\"" + LINK_PREFIX + "http://a.b.com/some.mov\"/>\n"
+        + "<link href=\"" + LINK_PREFIX + "http://a.b.com/link.html\"/>";
+    assertEquals(rewritten, rewriteHelper(s, p, null, pfxLinkRewriter));
+  }
+
+  // Tests style-tag rewriting
+  public void testStyleTagRewrites() throws Exception {
+    String css =
+      "div {list-style-image:url('http://a.b.com/bullet.gif');list-style-position:outside;margin:5px;padding:0}\n" +
+      ".someid {background-image:url(http://a.b.com/bigimg.png);float:right;width:165px;height:23px;margin-top:4px;margin-left:5px}";
+    String s = "<style>" + css + "</style>";
+    ParsedHtmlNode[] styleKids = {
+      GadgetHtmlNodeTest.makeParsedTextNode(css)
+    };
+    ParsedHtmlNode[] p = {
+      GadgetHtmlNodeTest.makeParsedTagNode("style", null, styleKids)
+    };
+    String rewritten =
+      "<style>div {list-style-image:url(\"" + LINK_PREFIX + "http://a.b.com/bullet.gif\");list-style-position:outside;margin:5px;padding:0}\n" +
+      ".someid {background-image:url(\"" + LINK_PREFIX + "http://a.b.com/bigimg.png\");float:right;width:165px;height:23px;margin-top:4px;margin-left:5px}</style>";
+    assertEquals(rewritten, rewriteHelper(s, p, styleFeature, pfxLinkRewriter));
+  }
+}