You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tika.apache.org by ju...@apache.org on 2008/01/21 21:29:33 UTC

svn commit: r614000 - in /incubator/tika/trunk: ./ src/main/java/org/apache/tika/parser/opendocument/ src/main/java/org/apache/tika/parser/xml/ src/main/java/org/apache/tika/sax/ src/main/java/org/apache/tika/sax/xpath/ src/test/java/org/apache/tika/pa...

Author: jukka
Date: Mon Jan 21 12:29:31 2008
New Revision: 614000

URL: http://svn.apache.org/viewvc?rev=614000&view=rev
Log:
TIKA-116: Streaming parser for OpenDocument files
    - Streaming XPath implementation in o.a.tika.sax.xpath
    - New o.a.tika.sax utility classes
    - Streaming XML parser
    - Avoid closing the input stream while parsing XML
    - Streaming OpenDocument parser
    - Extract correct OpenDocument MIME type while parsing

Added:
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeContentParser.java
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeMetaParser.java
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/MetadataHandler.java
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParser.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/TextContentHandler.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/AttributeMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ChildMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/CompositeMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ElementMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/Matcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/MatchingContentHandler.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedAttributeMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedElementMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/SubtreeMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/TextMatcher.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/XPathParser.java
    incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/
    incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/OpenOfficeParserTest.java
    incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/
    incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/DcXMLParserTest.java
    incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/
    incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/XPathParserTest.java
Removed:
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeEntityResolver.java
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParserUtils.java
Modified:
    incubator/tika/trunk/CHANGES.txt
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeParser.java
    incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/DcXMLParser.java
    incubator/tika/trunk/src/main/java/org/apache/tika/sax/ContentHandlerDecorator.java

Modified: incubator/tika/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/CHANGES.txt?rev=614000&r1=613999&r2=614000&view=diff
==============================================================================
--- incubator/tika/trunk/CHANGES.txt (original)
+++ incubator/tika/trunk/CHANGES.txt Mon Jan 21 12:29:31 2008
@@ -7,6 +7,8 @@
 2. TIKA-105 - Excel parser implementation based on POI's Event API
               (Niall Pemberton)
 
+3. TIKA-116 - Streaming parser for OpenDocument files (Jukka Zitting)
+
 Release 0.1-incubating - 12/27/2007
 
 1. TIKA-5 - Port Metadata Framework from Nutch (mattmann)

Added: incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeContentParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeContentParser.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeContentParser.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeContentParser.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,40 @@
+/**
+ * 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.tika.parser.opendocument;
+
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.parser.xml.XMLParser;
+import org.apache.tika.sax.xpath.MatchingContentHandler;
+import org.apache.tika.sax.xpath.XPathParser;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * Parser for OpenDocument <code>content.xml</code> files.
+ */
+public class OpenOfficeContentParser extends XMLParser {
+
+    private static final XPathParser OFFICE_XPATH = new XPathParser(
+            "office", "urn:oasis:names:tc:opendocument:xmlns:office:1.0");
+
+    protected DefaultHandler getDefaultHandler(ContentHandler ch, Metadata md) {
+        return new MatchingContentHandler(
+                super.getDefaultHandler(ch, md),
+                OFFICE_XPATH.parse("//office:body//text()"));
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeMetaParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeMetaParser.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeMetaParser.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeMetaParser.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,71 @@
+/**
+ * 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.tika.parser.opendocument;
+
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.parser.xml.DcXMLParser;
+import org.apache.tika.parser.xml.MetadataHandler;
+import org.apache.tika.sax.TeeContentHandler;
+import org.apache.tika.sax.xpath.CompositeMatcher;
+import org.apache.tika.sax.xpath.Matcher;
+import org.apache.tika.sax.xpath.MatchingContentHandler;
+import org.apache.tika.sax.xpath.XPathParser;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * Parser for OpenDocument <code>meta.xml</code> files.
+ */
+public class OpenOfficeMetaParser extends DcXMLParser {
+
+    private static final XPathParser META_XPATH = new XPathParser(
+            "meta", "urn:oasis:names:tc:opendocument:xmlns:meta:1.0");
+
+    private static DefaultHandler getMeta(
+            ContentHandler ch, Metadata md, String name, String element) {
+        Matcher matcher = new CompositeMatcher(
+                META_XPATH.parse("//meta:" + element),
+                META_XPATH.parse("//meta:" + element + "//text()"));
+        ContentHandler branch =
+            new MatchingContentHandler(new MetadataHandler(md, name), matcher);
+        return new TeeContentHandler(ch, branch);
+    }
+
+    private static DefaultHandler getStatistic(
+            ContentHandler ch, Metadata md, String name, String attribute) {
+        Matcher matcher =
+            META_XPATH.parse("//meta:document-statistic/@meta:" + attribute);
+        ContentHandler branch =
+            new MatchingContentHandler(new MetadataHandler(md, name), matcher);
+        return new TeeContentHandler(ch, branch);
+    }
+
+    protected DefaultHandler getDefaultHandler(ContentHandler ch, Metadata md) {
+        DefaultHandler dh = super.getDefaultHandler(ch, md);
+        dh = getMeta(dh, md, Metadata.KEYWORDS, "keyword");
+        dh = getMeta(dh, md, "generator", "generator");
+        dh = getStatistic(dh, md, "nbTab", "table-count");
+        dh = getStatistic(dh, md, "nbObject", "object-count");
+        dh = getStatistic(dh, md, "nbImg", "image-count");
+        dh = getStatistic(dh, md, "nbPage", "page-count");
+        dh = getStatistic(dh, md, "nbPara", "paragraph-count");
+        dh = getStatistic(dh, md, "nbWord", "word-count");
+        dh = getStatistic(dh, md, "nbCharacter", "character-count");
+        return dh;
+    }
+
+}

Modified: incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeParser.java?rev=614000&r1=613999&r2=614000&view=diff
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeParser.java (original)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/opendocument/OpenOfficeParser.java Mon Jan 21 12:29:31 2008
@@ -17,148 +17,65 @@
 
 package org.apache.tika.parser.opendocument;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
+import java.io.InputStreamReader;
+import java.io.Reader;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
-import org.apache.log4j.Logger;
 import org.apache.tika.exception.TikaException;
 import org.apache.tika.metadata.Metadata;
 import org.apache.tika.parser.Parser;
-import org.apache.tika.parser.xml.XMLParserUtils;
-import org.apache.tika.sax.AppendableAdaptor;
-import org.apache.tika.sax.XHTMLContentHandler;
-import org.jaxen.SimpleNamespaceContext;
-import org.jdom.Document;
-import org.jdom.Element;
-import org.jdom.JDOMException;
-import org.jdom.Namespace;
-import org.jdom.input.SAXBuilder;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
 
 /**
  * OpenOffice parser
  */
-public class OpenOfficeParser extends XMLParserUtils implements Parser {
-    static Logger logger = Logger.getRootLogger();
+public class OpenOfficeParser implements Parser {
 
-    private final Namespace NS_DC = Namespace.getNamespace("dc",
-            "http://purl.org/dc/elements/1.1/");
+    private Parser meta = new OpenOfficeMetaParser();
 
-    public org.jdom.Document parse(InputStream is) {
-        Document xmlDoc = new org.jdom.Document();
-        org.jdom.Document xmlMeta = new org.jdom.Document();
-        try {
-            List files = unzip(is);
-            SAXBuilder builder = new SAXBuilder();
-            builder.setEntityResolver(new OpenOfficeEntityResolver());
-            builder.setValidation(false);
-
-            xmlDoc = builder.build((InputStream) files.get(0));
-            xmlMeta = builder.build((InputStream) files.get(1));
-            Element rootMeta = xmlMeta.getRootElement();
-            Element meta = null;
-            List ls = rootMeta.getChildren();
-            if (!ls.isEmpty()) {
-                meta = (Element) ls.get(0);
-            }
-            xmlDoc.getRootElement().addContent(meta.detach());
-            xmlDoc.getRootElement().addNamespaceDeclaration(NS_DC);
-        } catch (JDOMException e) {
-            logger.error(e.getMessage());
-        } catch (IOException e) {
-            logger.error(e.getMessage());
-        }
-        return xmlDoc;
-    }
+    private Parser content = new OpenOfficeContentParser();
 
-    public void parse(InputStream stream, ContentHandler handler,
-            Metadata metadata) throws IOException, SAXException, TikaException {
-        Document xmlDoc = parse(stream);
-        // Set NameSpaceContext for OpenDocument
-        SimpleNamespaceContext context = new SimpleNamespaceContext();
-        context.addNamespace("dc", "http://purl.org/dc/elements/1.1/");
-        context.addNamespace("meta",
-                "urn:oasis:names:tc:opendocument:xmlns:meta:1.0");
-        context.addNamespace("office",
-                "urn:oasis:names:tc:opendocument:xmlns:office:1.0");
-        setXmlParserNameSpaceContext(context);
-
-        extractContent(xmlDoc, Metadata.TITLE, "//dc:title", metadata);
-        extractContent(xmlDoc, Metadata.SUBJECT, "//dc:subject", metadata);
-        extractContent(xmlDoc, Metadata.CREATOR, "//dc:creator", metadata);
-        extractContent(xmlDoc, Metadata.DESCRIPTION, "//dc:description",
-                metadata);
-        extractContent(xmlDoc, Metadata.LANGUAGE, "//dc:language", metadata);
-        extractContent(xmlDoc, Metadata.KEYWORDS, "//meta:keyword", metadata);
-        extractContent(xmlDoc, Metadata.DATE, "//dc:date", metadata);
-        extractContent(xmlDoc, "nbTab",
-                "//meta:document-statistic/@meta:table-count", metadata);
-        extractContent(xmlDoc, "nbObject",
-                "//meta:document-statistic/@meta:object-count", metadata);
-        extractContent(xmlDoc, "nbImg",
-                "//meta:document-statistic/@meta:image-count", metadata);
-        extractContent(xmlDoc, "nbPage",
-                "//meta:document-statistic/@meta:page-count", metadata);
-        extractContent(xmlDoc, "nbPara",
-                "//meta:document-statistic/@meta:paragraph-count", metadata);
-        extractContent(xmlDoc, "nbWord",
-                "//meta:document-statistic/@meta:word-count", metadata);
-        extractContent(xmlDoc, "nbcharacter",
-                "//meta:document-statistic/@meta:character-count", metadata);
-
-        XHTMLContentHandler xhtml = new XHTMLContentHandler(handler, metadata);
-        xhtml.startDocument();
-        xhtml.startElement("p");
-        concatOccurrence(xmlDoc, "//office:body//*", " ",
-                new AppendableAdaptor(xhtml));
-        xhtml.endElement("p");
-        xhtml.endDocument();
+    public Parser getMetaParser() {
+        return meta;
     }
 
-    public List unzip(InputStream is) {
-        List res = new ArrayList();
-        try {
-            ZipInputStream in = new ZipInputStream(is);
-            ZipEntry entry = null;
-            while ((entry = in.getNextEntry()) != null) {
-                if (entry.getName().equals("meta.xml")
-                        || entry.getName().equals("content.xml")) {
-                    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-                    byte[] buf = new byte[1024];
-                    int len;
-                    while ((len = in.read(buf)) > 0) {
-                        stream.write(buf, 0, len);
-                    }
-                    InputStream isEntry = new ByteArrayInputStream(stream
-                            .toByteArray());
-                    res.add(isEntry);
-                }
-            }
-            in.close();
-        } catch (IOException e) {
-            logger.error(e.getMessage());
-        }
-        return res;
+    public void setMetaParser(Parser meta) {
+        this.meta = meta;
     }
 
-    protected void copyInputStream(InputStream in, OutputStream out)
-            throws IOException {
-        byte[] buffer = new byte[1024];
-        int len;
+    public Parser getContentParser() {
+        return content;
+    }
 
-        while ((len = in.read(buffer)) >= 0)
-            out.write(buffer, 0, len);
+    public void setContentParser(Parser content) {
+        this.content = content;
+    }
 
-        in.close();
-        out.close();
+    public void parse(
+            InputStream stream, ContentHandler handler, Metadata metadata)
+            throws IOException, SAXException, TikaException {
+        ZipInputStream zip = new ZipInputStream(stream);
+        ZipEntry entry = zip.getNextEntry();
+        while (entry != null) {
+            if (entry.getName().equals("mimetype")) {
+                StringBuilder buffer = new StringBuilder();
+                Reader reader = new InputStreamReader(zip, "UTF-8");
+                for (int ch = reader.read(); ch != -1; ch = reader.read()) {
+                    buffer.append((char) ch);
+                }
+                metadata.set(Metadata.CONTENT_TYPE, buffer.toString());
+            } else if (entry.getName().equals("meta.xml")) {
+                meta.parse(zip, new DefaultHandler(), metadata);
+            } else if (entry.getName().equals("content.xml")) {
+                content.parse(zip, handler, metadata);
+            }
+            entry = zip.getNextEntry();
+        }
     }
 
 }

Modified: incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/DcXMLParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/DcXMLParser.java?rev=614000&r1=613999&r2=614000&view=diff
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/DcXMLParser.java (original)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/DcXMLParser.java Mon Jan 21 12:29:31 2008
@@ -16,51 +16,49 @@
  */
 package org.apache.tika.parser.xml;
 
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.tika.exception.TikaException;
+import org.apache.tika.metadata.DublinCore;
 import org.apache.tika.metadata.Metadata;
-import org.apache.tika.parser.Parser;
-import org.apache.tika.sax.AppendableAdaptor;
-import org.apache.tika.sax.XHTMLContentHandler;
-import org.apache.tika.utils.Utils;
-import org.jaxen.SimpleNamespaceContext;
-import org.jdom.Document;
+import org.apache.tika.sax.TeeContentHandler;
+import org.apache.tika.sax.xpath.CompositeMatcher;
+import org.apache.tika.sax.xpath.Matcher;
+import org.apache.tika.sax.xpath.MatchingContentHandler;
+import org.apache.tika.sax.xpath.XPathParser;
 import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
 
 /**
- * Dublin core metadata parser
+ * Dublin Core metadata parser
  */
-public class DcXMLParser extends XMLParserUtils implements Parser {
+public class DcXMLParser extends XMLParser {
+
+    private static final XPathParser DC_XPATH = new XPathParser(
+            "dc", "http://purl.org/dc/elements/1.1/");
 
-    public void parse(InputStream stream, ContentHandler handler,
-            Metadata metadata) throws IOException, SAXException, TikaException {
-        Document xmlDoc = Utils.parse(stream);
-        // Set NameSpaceContext for Dublin Core metadata
-        SimpleNamespaceContext context = new SimpleNamespaceContext();
-        context.addNamespace("dc", "http://purl.org/dc/elements/1.1/");
-        setXmlParserNameSpaceContext(context);
-        extractContent(xmlDoc, Metadata.TITLE, "//dc:title", metadata);
-        extractContent(xmlDoc, Metadata.SUBJECT, "//dc:subject", metadata);
-        extractContent(xmlDoc, Metadata.CREATOR, "//dc:creator", metadata);
-        extractContent(xmlDoc, Metadata.DESCRIPTION, "//dc:description",
-                metadata);
-        extractContent(xmlDoc, Metadata.PUBLISHER, "//dc:publisher", metadata);
-        extractContent(xmlDoc, Metadata.CONTRIBUTOR, "//dc:contributor",
-                metadata);
-        extractContent(xmlDoc, Metadata.TYPE, "//dc:type", metadata);
-        extractContent(xmlDoc, Metadata.FORMAT, "//dc:format", metadata);
-        extractContent(xmlDoc, Metadata.IDENTIFIER, "//dc:identifier", metadata);
-        extractContent(xmlDoc, Metadata.LANGUAGE, "//dc:language", metadata);
-        extractContent(xmlDoc, Metadata.RIGHTS, "//dc:rights", metadata);
+    private static DefaultHandler getDublinCore(
+            ContentHandler ch, Metadata md, String name, String element) {
+        Matcher matcher = new CompositeMatcher(
+                DC_XPATH.parse("//dc:" + element),
+                DC_XPATH.parse("//dc:" + element + "//text()"));
+        ContentHandler branch =
+            new MatchingContentHandler(new MetadataHandler(md, name), matcher);
+        return new TeeContentHandler(ch, branch);
+    }
 
-        XHTMLContentHandler xhtml = new XHTMLContentHandler(handler, metadata);
-        xhtml.startDocument();
-        xhtml.startElement("p");
-        concatOccurrence(xmlDoc, "//*", " ", new AppendableAdaptor(xhtml));
-        xhtml.endElement("p");
-        xhtml.endDocument();
+    protected DefaultHandler getDefaultHandler(ContentHandler ch, Metadata md) {
+        DefaultHandler dh = super.getDefaultHandler(ch, md);
+        dh = getDublinCore(dh, md, DublinCore.TITLE, "title");
+        dh = getDublinCore(dh, md, DublinCore.SUBJECT, "subject");
+        dh = getDublinCore(dh, md, DublinCore.CREATOR, "creator");
+        dh = getDublinCore(dh, md, DublinCore.DESCRIPTION, "description");
+        dh = getDublinCore(dh, md, DublinCore.PUBLISHER, "publisher");
+        dh = getDublinCore(dh, md, DublinCore.CONTRIBUTOR, "contributor");
+        dh = getDublinCore(dh, md, DublinCore.DATE, "date");
+        dh = getDublinCore(dh, md, DublinCore.TYPE, "type");
+        dh = getDublinCore(dh, md, DublinCore.FORMAT, "format");
+        dh = getDublinCore(dh, md, DublinCore.IDENTIFIER, "identifier");
+        dh = getDublinCore(dh, md, DublinCore.LANGUAGE, "language");
+        dh = getDublinCore(dh, md, DublinCore.RIGHTS, "rights");
+        return dh;
     }
+
 }

Added: incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/MetadataHandler.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/MetadataHandler.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/MetadataHandler.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/MetadataHandler.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,63 @@
+/*
+ * 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.tika.parser.xml;
+
+import org.apache.tika.metadata.Metadata;
+import org.xml.sax.Attributes;
+import org.xml.sax.helpers.DefaultHandler;
+
+public class MetadataHandler extends DefaultHandler {
+
+    private final Metadata metadata;
+
+    private final String name;
+
+    private final StringBuilder buffer = new StringBuilder();
+
+    public MetadataHandler(Metadata metadata, String name) {
+        this.metadata = metadata;
+        this.name = name;
+    }
+
+    public void addMetadata(String value) {
+        if (value.length() > 0) {
+            String previous = metadata.get(name);
+            if (previous != null && previous.length() > 0) {
+                value = previous + ", " + value;
+            }
+            metadata.set(name, value);
+        }
+    }
+
+    public void endElement(String uri, String localName, String name) {
+        addMetadata(buffer.toString());
+        buffer.setLength(0);
+    }
+
+    public void startElement(
+            String uri, String localName, String name, Attributes attributes) {
+        for (int i = 0; i < attributes.getLength(); i++) {
+            addMetadata(attributes.getValue(i));
+        }
+    }
+
+    
+    public void characters(char[] ch, int start, int length) {
+        buffer.append(ch, start, length);
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParser.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParser.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/parser/xml/XMLParser.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,74 @@
+/**
+ * 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.tika.parser.xml;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.parser.Parser;
+import org.apache.tika.sax.TextContentHandler;
+import org.apache.tika.sax.XHTMLContentHandler;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * XML parser
+ */
+public class XMLParser implements Parser {
+
+    public void parse(
+            InputStream stream, ContentHandler handler, Metadata metadata)
+            throws IOException, SAXException, TikaException {
+        if (metadata.get(Metadata.CONTENT_TYPE) == null) {
+            metadata.set(Metadata.CONTENT_TYPE, "application/xml");
+        }
+
+        final XHTMLContentHandler xhtml =
+            new XHTMLContentHandler(handler, metadata);
+        xhtml.startDocument();
+        xhtml.startElement("p");
+
+        try {
+            SAXParserFactory factory = SAXParserFactory.newInstance();
+            factory.setNamespaceAware(true);
+            SAXParser parser = factory.newSAXParser();
+            stream = new BufferedInputStream(stream) {
+                public void close() {}
+            };
+            parser.parse(stream, getDefaultHandler(handler, metadata));
+        } catch (ParserConfigurationException e) {
+            throw new TikaException("XML parser configuration error", e);
+        }
+
+        xhtml.endElement("p");
+        xhtml.endDocument();
+    }
+
+    protected DefaultHandler getDefaultHandler(
+            ContentHandler handler, Metadata metadata) {
+        return new TextContentHandler(handler);
+    }
+
+}

Modified: incubator/tika/trunk/src/main/java/org/apache/tika/sax/ContentHandlerDecorator.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/ContentHandlerDecorator.java?rev=614000&r1=613999&r2=614000&view=diff
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/ContentHandlerDecorator.java (original)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/ContentHandlerDecorator.java Mon Jan 21 12:29:31 2008
@@ -20,6 +20,7 @@
 import org.xml.sax.ContentHandler;
 import org.xml.sax.Locator;
 import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
 
 /**
  * Decorator base class for the {@link ContentHandler} interface. This class
@@ -27,7 +28,7 @@
  * instance. Subclasses can provide extra decoration by overriding one or more
  * of the SAX event methods.
  */
-public class ContentHandlerDecorator implements ContentHandler {
+public class ContentHandlerDecorator extends DefaultHandler {
 
     /**
      * Decorated SAX event handler.

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/TextContentHandler.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/TextContentHandler.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/TextContentHandler.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/TextContentHandler.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,47 @@
+/*
+ * 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.tika.sax;
+
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * Content handler decorator that only passes the
+ * {@link #characters(char[], int, int)} and
+ * (@link {@link #ignorableWhitespace(char[], int, int)} events to
+ * the decorated content handler.
+ */
+public class TextContentHandler extends DefaultHandler {
+
+    private final ContentHandler delegate;
+
+    public TextContentHandler(ContentHandler delegate) {
+        this.delegate = delegate;
+    }
+
+    public void characters(char[] ch, int start, int length)
+            throws SAXException {
+        delegate.characters(ch, start, length);
+    }
+
+    public void ignorableWhitespace(char[] ch, int start, int length)
+            throws SAXException {
+        delegate.ignorableWhitespace(ch, start, length);
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/AttributeMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/AttributeMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/AttributeMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/AttributeMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,31 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Final evaluation state of a <code>.../@*</code> XPath expression.
+ * Matches all attributes of the current element.
+ */
+public class AttributeMatcher extends Matcher {
+
+    public static final Matcher INSTANCE = new AttributeMatcher();
+
+    public boolean matchesAttribute(String namespace, String name) {
+        return true;
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ChildMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ChildMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ChildMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ChildMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,35 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Intermediate evaluation state of a <code>.../*...</code> XPath expression.
+ * Matches nothing, but specifies the evaluation state for all child elements.
+ */
+public class ChildMatcher extends Matcher {
+
+    private final Matcher then;
+
+    public ChildMatcher(Matcher then) {
+        this.then = then;
+    }
+
+    public Matcher descend(String namespace, String name) {
+        return then;
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/CompositeMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/CompositeMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/CompositeMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/CompositeMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,61 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Composite XPath evaluation state. Used when XPath evaluation results
+ * in two or more branches of independent evaluation states.
+ */
+public class CompositeMatcher extends Matcher {
+
+    private final Matcher a;
+
+    private final Matcher b;
+
+    public CompositeMatcher(Matcher a, Matcher b) {
+        this.a = a;
+        this.b = b;
+    }
+
+    public Matcher descend(String namespace, String name) {
+        Matcher a = this.a.descend(namespace, name);
+        Matcher b = this.b.descend(namespace, name);
+        if (a == FAIL) {
+            return b;
+        } else if (b == FAIL) {
+            return a;
+        } else if (this.a == a && this.b == b) {
+            return this;
+        } else {
+            return new CompositeMatcher(a, b);
+        }
+    }
+
+    public boolean matchesElement() {
+        return a.matchesElement() || b.matchesElement();
+    }
+
+    public boolean matchesAttribute(String namespace, String name) {
+        return a.matchesAttribute(namespace, name)
+            || b.matchesAttribute(namespace, name);
+    }
+
+    public boolean matchesText() {
+        return a.matchesText() || b.matchesText();
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ElementMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ElementMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ElementMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/ElementMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,31 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Final evaluation state of an XPath expression that targets an element.
+ * Matches the current element.
+ */
+public class ElementMatcher extends Matcher {
+
+    public static final Matcher INSTANCE = new ElementMatcher();
+
+    public boolean matchesElement() {
+        return true;
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/Matcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/Matcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/Matcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/Matcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,77 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * XPath element matcher. A matcher instance encapsulates a specific
+ * state in XPath evaluation.
+ */
+public class Matcher {
+
+    /**
+     * State of a failed XPath evaluation, where nothing is matched.
+     * This matcher instance is used as a sentinel object whenever an
+     * XPath evaluation branch fails.
+     */
+    public static final Matcher FAIL = new Matcher();
+
+    /**
+     * Returns the XPath evaluation state that results from descending
+     * to a child element with the given name.
+     *
+     * @param namespace element namespace or <code>null</code>
+     * @param name element name
+     * @return next XPath evaluation state
+     */
+    public Matcher descend(String namespace, String name) {
+        return FAIL;
+    }
+
+    /**
+     * Returns <code>true</code> if the XPath expression matches
+     * the element associated with this evaluation state.
+     *
+     * @return XPath evaluation state for this element
+     */
+    public boolean matchesElement() {
+        return false;
+    }
+
+    /**
+     * Returns <code>true</code> if the XPath expression matches the named
+     * attribute of the element associated with this evaluation state.
+     *
+     * @param namespace attribute namespace or <code>null</code>
+     * @param name attribute name
+     * @return XPath evaluation state for named attribute of this element
+     */
+    public boolean matchesAttribute(String namespace, String name) {
+        return false;
+    }
+
+    /**
+     * Returns <code>true</code> if the XPath expression matches all text
+     * nodes whose parent is the element associated with this evaluation
+     * state.
+     *
+     * @return XPath evaluation state for text children of this element
+     */
+    public boolean matchesText() {
+        return false;
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/MatchingContentHandler.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/MatchingContentHandler.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/MatchingContentHandler.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/MatchingContentHandler.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,103 @@
+/*
+ * 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.tika.sax.xpath;
+
+import java.util.LinkedList;
+
+import org.apache.tika.sax.ContentHandlerDecorator;
+import org.xml.sax.Attributes;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
+/**
+ * Content handler decorator that only passes the elements, attributes,
+ * and text nodes that match the given XPath expression.
+ */
+public class MatchingContentHandler extends ContentHandlerDecorator {
+
+    private final LinkedList<Matcher> matchers = new LinkedList<Matcher>();
+
+    private Matcher matcher;
+
+    public MatchingContentHandler(ContentHandler delegate, Matcher matcher) {
+        super(delegate);
+        this.matcher = matcher;
+    }
+
+    public void startElement(
+            String uri, String localName, String name, Attributes attributes)
+            throws SAXException {
+        matchers.addFirst(matcher);
+        matcher = matcher.descend(uri, localName);
+
+        AttributesImpl matches = new AttributesImpl();
+        for (int i = 0; i < attributes.getLength(); i++) {
+            String attributeURI = attributes.getURI(i);
+            String attributeName = attributes.getLocalName(i);
+            if (matcher.matchesAttribute(attributeURI, attributeName)) {
+                matches.addAttribute(
+                        attributeURI, attributeName, attributes.getQName(i),
+                        attributes.getType(i), attributes.getValue(i));
+            }
+        }
+
+        if (matcher.matchesElement() || matches.getLength() > 0) {
+            super.startElement(uri, localName, name, matches);
+            if (!matcher.matchesElement()) {
+                // Force the matcher to match the current element, so the
+                // endElement method knows to emit the correct event
+                matcher =
+                    new CompositeMatcher(matcher, ElementMatcher.INSTANCE);
+            }
+        }
+    }
+
+    public void endElement(String uri, String localName, String name)
+            throws SAXException {
+        if (matcher.matchesElement()) {
+            super.endElement(uri, localName, name);
+        }
+        matcher = matchers.removeFirst();
+    }
+
+    public void characters(char[] ch, int start, int length)
+            throws SAXException {
+        if (matcher.matchesText()) {
+            super.characters(ch, start, length);
+        }
+    }
+
+    public void ignorableWhitespace(char[] ch, int start, int length)
+            throws SAXException {
+        if (matcher.matchesText()) {
+            super.ignorableWhitespace(ch, start, length);
+        }
+    }
+
+    public void processingInstruction(String target, String data) {
+        // TODO: Support for matching processing instructions
+    }
+
+    public void skippedEntity(String name) throws SAXException {
+        // TODO: Can skipped entities refer to more than text?
+        if (matcher.matchesText()) {
+            super.skippedEntity(name);
+        }
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedAttributeMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedAttributeMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedAttributeMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedAttributeMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,42 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Final evaluation state of a <code>.../@name</code> XPath expression.
+ * Matches the named attributes of the current element.
+ */
+public class NamedAttributeMatcher extends Matcher {
+
+    private final String namespace;
+
+    private final String name;
+
+    public NamedAttributeMatcher(String namespace, String name) {
+        this.namespace = namespace;
+        this.name = name;
+    }
+
+    public boolean matchesAttribute(String namespace, String name) {
+        return equals(namespace, this.namespace) && name.equals(this.name);
+    }
+
+    private static boolean equals(String a, String b) {
+        return (a == null) ? (b == null) : a.equals(b);
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedElementMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedElementMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedElementMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/NamedElementMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,48 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Intermediate evaluation state of a <code>.../name...</code> XPath
+ * expression. Matches nothing, but specifies the evaluation state
+ * for the child elements with the given name.
+ */
+public class NamedElementMatcher extends ChildMatcher {
+
+    private final String namespace;
+
+    private final String name;
+
+    protected NamedElementMatcher(String namespace, String name, Matcher then) {
+        super(then);
+        this.namespace = namespace;
+        this.name = name;
+    }
+
+    public Matcher descend(String namespace, String name) {
+        if (equals(namespace, this.namespace) && name.equals(this.name)) {
+            return super.descend(namespace, name);
+        } else {
+            return FAIL;
+        }
+    }
+
+    private static boolean equals(String a, String b) {
+        return (a == null) ? (b == null) : a.equals(b);
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/SubtreeMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/SubtreeMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/SubtreeMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/SubtreeMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,48 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Evaluation state of a <code>...//...</code> XPath expression. Applies the
+ * contained evaluation state to the current element and all its descendants.
+ */
+public class SubtreeMatcher extends Matcher {
+
+    private final Matcher then;
+
+    public SubtreeMatcher(Matcher then) {
+        this.then = then;
+    }
+
+    public Matcher descend(String namespace, String name) {
+        Matcher next = then.descend(namespace, name);
+        if (next == FAIL || next == then) {
+            return this;
+        } else {
+            return new CompositeMatcher(next, this);
+        }
+    }
+
+    public boolean matchesAttribute(String namespace, String name) {
+        return then.matchesAttribute(namespace, name);
+    }
+
+    public boolean matchesText() {
+        return then.matchesText();
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/TextMatcher.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/TextMatcher.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/TextMatcher.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/TextMatcher.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,31 @@
+/*
+ * 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.tika.sax.xpath;
+
+/**
+ * Final evaluation state of a <code>.../text()</code> XPath expression.
+ * Matches all text children of the current element.
+ */
+public class TextMatcher extends Matcher {
+
+    public static final Matcher INSTANCE = new TextMatcher();
+
+    public boolean matchesText() {
+        return true;
+    }
+
+}

Added: incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/XPathParser.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/XPathParser.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/XPathParser.java (added)
+++ incubator/tika/trunk/src/main/java/org/apache/tika/sax/xpath/XPathParser.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,108 @@
+/*
+ * 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.tika.sax.xpath;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Parser for a very simple XPath subset. Only the following XPath constructs
+ * (with namespaces) are supported:
+ * <ul>
+ *   <li><code>.../text()</code></li>
+ *   <li><code>.../@*</code></li>
+ *   <li><code>.../@name</code></li>
+ *   <li><code>.../*...</code></li>
+ *   <li><code>.../name...</code></li>
+ *   <li><code>...//*...</code></li>
+ *   <li><code>...//name...</code></li>
+ * </ul>
+ */
+public class XPathParser {
+
+    private final Map<String, String> prefixes = new HashMap<String, String>();
+
+    public XPathParser() {
+    }
+
+    public XPathParser(String prefix, String namespace) {
+        addPrefix(prefix, namespace);
+    }
+
+    public void addPrefix(String prefix, String namespace) {
+        prefixes.put(prefix, namespace);
+    }
+
+    /**
+     * Parses the given simple XPath expression to an evaluation state
+     * initialized at the document node. Invalid expressions are not flagged
+     * as errors, they just result in a failing evaluation state.
+     *
+     * @param xpath simple XPath expression
+     * @return XPath evaluation state
+     */
+    public Matcher parse(String xpath) {
+        if (xpath.equals("/text()")) {
+            return TextMatcher.INSTANCE;
+        } else if (xpath.equals("/@*")) {
+            return AttributeMatcher.INSTANCE;
+        } else if (xpath.length() == 0) {
+            return ElementMatcher.INSTANCE;
+        } else if (xpath.startsWith("/@")) {
+            String name = xpath.substring(2);
+            String prefix = null;
+            int colon = name.indexOf(':');
+            if (colon != -1) {
+                prefix = name.substring(0, colon);
+                name = name.substring(colon + 1);
+            }
+            if (prefixes.containsKey(prefix)) {
+                return new NamedAttributeMatcher(prefixes.get(prefix), name);
+            } else {
+                return Matcher.FAIL;
+            }
+        } else if (xpath.startsWith("/*")) {
+            return new ChildMatcher(parse(xpath.substring(2)));
+        } else if (xpath.startsWith("///")) {
+            return Matcher.FAIL;
+        } else if (xpath.startsWith("//")) {
+            return new SubtreeMatcher(parse(xpath.substring(1)));
+        } else if (xpath.startsWith("/")) {
+            int slash = xpath.indexOf('/', 1);
+            if (slash == -1) {
+                slash = xpath.length();
+            }
+            String name = xpath.substring(1, slash);
+            String prefix = null;
+            int colon = name.indexOf(':');
+            if (colon != -1) {
+                prefix = name.substring(0, colon);
+                name = name.substring(colon + 1);
+            }
+            if (prefixes.containsKey(prefix)) {
+                return new NamedElementMatcher(
+                        prefixes.get(prefix), name,
+                        parse(xpath.substring(slash)));
+            } else {
+                return Matcher.FAIL;
+            }
+        } else {
+            return Matcher.FAIL;
+        }
+    }
+
+}

Added: incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/OpenOfficeParserTest.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/OpenOfficeParserTest.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/OpenOfficeParserTest.java (added)
+++ incubator/tika/trunk/src/test/java/org/apache/tika/parser/opendocument/OpenOfficeParserTest.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,64 @@
+/**
+ * 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.tika.parser.opendocument;
+
+import java.io.InputStream;
+import java.io.StringWriter;
+
+import junit.framework.TestCase;
+
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.sax.WriteOutContentHandler;
+import org.xml.sax.ContentHandler;
+
+public class OpenOfficeParserTest extends TestCase {
+
+    public void testXMLParser() throws Exception {
+        InputStream input = OpenOfficeParserTest.class.getResourceAsStream(
+                "/test-documents/testOpenOffice2.odt");
+        try {
+            Metadata metadata = new Metadata();
+            StringWriter writer = new StringWriter();
+            ContentHandler handler = new WriteOutContentHandler(writer);
+            new OpenOfficeParser().parse(input, handler, metadata);
+
+            assertEquals(
+                    "application/vnd.oasis.opendocument.text",
+                    metadata.get(Metadata.CONTENT_TYPE));
+            assertEquals("2007-09-14T11:07:10", metadata.get(Metadata.DATE));
+            assertEquals("en-US", metadata.get(Metadata.LANGUAGE));
+            assertEquals(
+                    "NeoOffice/2.2$Unix OpenOffice.org_project/680m18$Build-9161",
+                    metadata.get("generator"));
+            assertEquals("0", metadata.get("nbTab"));
+            assertEquals("0", metadata.get("nbObject"));
+            assertEquals("0", metadata.get("nbImg"));
+            assertEquals("1", metadata.get("nbPage"));
+            assertEquals("1", metadata.get("nbPara"));
+            assertEquals("14", metadata.get("nbWord"));
+            assertEquals("78", metadata.get("nbCharacter"));
+
+            String content = writer.toString();
+            assertTrue(content.contains(
+                    "This is a sample Open Office document,"
+                    + " written in NeoOffice 2.2.1 for the Mac."));
+        } finally {
+            input.close();
+        }
+    }
+
+}

Added: incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/DcXMLParserTest.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/DcXMLParserTest.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/DcXMLParserTest.java (added)
+++ incubator/tika/trunk/src/test/java/org/apache/tika/parser/xml/DcXMLParserTest.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,65 @@
+/**
+ * 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.tika.parser.xml;
+
+import java.io.InputStream;
+import java.io.StringWriter;
+
+import junit.framework.TestCase;
+
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.sax.WriteOutContentHandler;
+import org.xml.sax.ContentHandler;
+
+public class DcXMLParserTest extends TestCase {
+
+    public void testXMLParser() throws Exception {
+        InputStream input = DcXMLParserTest.class.getResourceAsStream(
+                "/test-documents/testXML.xml");
+        try {
+            Metadata metadata = new Metadata();
+            StringWriter writer = new StringWriter();
+            ContentHandler handler = new WriteOutContentHandler(writer);
+            new DcXMLParser().parse(input, handler, metadata);
+
+            assertEquals(
+                    "application/xml",
+                    metadata.get(Metadata.CONTENT_TYPE));
+            assertEquals("Archimède et Lius", metadata.get(Metadata.TITLE));
+            assertEquals("Rida Benjelloun", metadata.get(Metadata.CREATOR));
+            assertEquals(
+                    "Java, XML, XSLT, JDOM, Indexation",
+                    metadata.get(Metadata.SUBJECT));
+            assertEquals(
+                    "Framework d\'indexation des documents XML, HTML, PDF etc.. ",
+                    metadata.get(Metadata.DESCRIPTION));
+            assertEquals(
+                    "http://www.apache.org",
+                    metadata.get(Metadata.IDENTIFIER));
+            assertEquals("test", metadata.get(Metadata.TYPE));
+            assertEquals("application/msword", metadata.get(Metadata.FORMAT));
+            assertEquals("Fr", metadata.get(Metadata.LANGUAGE));
+            assertEquals("Non restreint", metadata.get(Metadata.RIGHTS));
+
+            String content = writer.toString();
+            assertTrue(content.contains("Archimède et Lius"));
+        } finally {
+            input.close();
+        }
+    }
+
+}

Added: incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/XPathParserTest.java
URL: http://svn.apache.org/viewvc/incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/XPathParserTest.java?rev=614000&view=auto
==============================================================================
--- incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/XPathParserTest.java (added)
+++ incubator/tika/trunk/src/test/java/org/apache/tika/sax/xpath/XPathParserTest.java Mon Jan 21 12:29:31 2008
@@ -0,0 +1,121 @@
+/*
+ * 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.tika.sax.xpath;
+
+import junit.framework.TestCase;
+
+public class XPathParserTest extends TestCase {
+
+    private static final String NS = "test namespace";
+
+    private XPathParser parser;
+
+    protected void setUp() {
+        parser = new XPathParser();
+        parser.addPrefix(null, null);
+        parser.addPrefix("prefix", NS);
+    }
+
+    public void testText() {
+        Matcher matcher = parser.parse("/text()");
+        assertTrue(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+    }
+
+    public void testAnyAttribute() {
+        Matcher matcher = parser.parse("/@*");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertTrue(matcher.matchesAttribute(null, "name"));
+        assertTrue(matcher.matchesAttribute(NS, "name"));
+        assertTrue(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+    }
+
+    public void testNamedAttribute() {
+        Matcher matcher = parser.parse("/@name");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertTrue(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+    }
+
+    public void testPrefixedAttribute() {
+        Matcher matcher = parser.parse("/@prefix:name");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertTrue(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+    }
+
+    public void testAnyElement() {
+        Matcher matcher = parser.parse("/*");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        matcher = matcher.descend(NS, "name");
+        assertFalse(matcher.matchesText());
+        assertTrue(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+    }
+
+    public void testNamedElement() {
+        Matcher matcher = parser.parse("/name");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "name"));
+        assertEquals(Matcher.FAIL, matcher.descend(null, "enam"));
+        matcher = matcher.descend(null, "name");
+        assertFalse(matcher.matchesText());
+        assertTrue(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+    }
+
+    public void testPrefixedElement() {
+        Matcher matcher = parser.parse("/prefix:name");
+        assertFalse(matcher.matchesText());
+        assertFalse(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+        assertEquals(Matcher.FAIL, matcher.descend(null, "name"));
+        assertEquals(Matcher.FAIL, matcher.descend(NS, "enam"));
+        matcher = matcher.descend(NS, "name");
+        assertFalse(matcher.matchesText());
+        assertTrue(matcher.matchesElement());
+        assertFalse(matcher.matchesAttribute(null, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "name"));
+        assertFalse(matcher.matchesAttribute(NS, "eman"));
+    }
+
+}