You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tika.apache.org by ni...@apache.org on 2010/09/09 13:55:32 UTC

svn commit: r995389 - in /tika/trunk: tika-core/src/main/java/org/apache/tika/extractor/ tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ tika-parsers/src/test/java/org/apache/tika/parser/microsoft/

Author: nick
Date: Thu Sep  9 11:55:32 2010
New Revision: 995389

URL: http://svn.apache.org/viewvc?rev=995389&view=rev
Log:
Refactor how container extraction works - Jukka's patch from TIKA-509
Replace the AutoContainerExtractor with ParserContainerExtractor, and push more of the work to the Parsers

Added:
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ParserContainerExtractor.java
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/AbstractPOIFSExtractor.java
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/WordExtractor.java
    tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIContainerExtractionTest.java
      - copied, changed from r995359, tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java
Removed:
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/POIFSContainerExtractor.java
    tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java
Modified:
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ExcelExtractor.java
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java

Added: tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ParserContainerExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ParserContainerExtractor.java?rev=995389&view=auto
==============================================================================
--- tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ParserContainerExtractor.java (added)
+++ tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ParserContainerExtractor.java Thu Sep  9 11:55:32 2010
@@ -0,0 +1,102 @@
+/**
+ * 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.extractor;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Set;
+
+import org.apache.tika.config.TikaConfig;
+import org.apache.tika.detect.Detector;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.io.TikaInputStream;
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.mime.MediaType;
+import org.apache.tika.parser.AutoDetectParser;
+import org.apache.tika.parser.ParseContext;
+import org.apache.tika.parser.Parser;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+/**
+ * An implementation of {@link ContainerExtractor} powered by the
+ *  regular {@link Parser} classes.
+ * This allows you to easily extract out all the embeded resources
+ *  from within contain files, whilst using the normal parsers
+ *  to do the work.
+ * By default the {@link AutoDetectParser} will be used, to allow
+ *  extraction from the widest range of containers.
+ */
+public class ParserContainerExtractor implements ContainerExtractor {
+    /**
+     * Serial version UID
+     */
+    private static final long serialVersionUID = 2261131045580861514L;
+
+    private final Parser parser;
+
+    private final Detector detector;
+
+    public ParserContainerExtractor() {
+        this(TikaConfig.getDefaultConfig());
+    }
+
+    public ParserContainerExtractor(TikaConfig config) {
+        this(new AutoDetectParser(config), config.getMimeRepository());
+    }
+
+    public ParserContainerExtractor(Parser parser, Detector detector) {
+        this.parser = parser;
+        this.detector = detector;
+    }
+
+    public boolean isSupported(TikaInputStream input) throws IOException {
+        MediaType type = detector.detect(input, new Metadata());
+        return parser.getSupportedTypes(new ParseContext()).contains(type);
+    }
+
+    public void extract(
+            TikaInputStream stream, ContainerExtractor recurseExtractor,
+            final ContainerEmbededResourceHandler handler)
+            throws IOException, TikaException {
+        ParseContext context = new ParseContext();
+        context.set(Parser.class, new Parser() {
+            public Set<MediaType> getSupportedTypes(ParseContext context) {
+                return parser.getSupportedTypes(context);
+            }
+            public void parse(InputStream stream, ContentHandler ignored,
+                    Metadata metadata, ParseContext context)
+                    throws IOException, SAXException, TikaException {
+                String filename = metadata.get(Metadata.RESOURCE_NAME_KEY);
+                MediaType type = detector.detect(stream, metadata);
+                handler.handle(filename, type, stream);
+            }
+            public void parse(InputStream stream, ContentHandler handler,
+                    Metadata metadata) throws IOException, SAXException,
+                    TikaException {
+                parse(stream, handler, metadata, new ParseContext());
+            }
+        });
+        try {
+            parser.parse(stream, new DefaultHandler(), new Metadata(), context);
+        } catch (SAXException e) {
+            throw new TikaException("Unexpected SAX exception", e);
+        }
+    }
+
+}

Added: tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/AbstractPOIFSExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/AbstractPOIFSExtractor.java?rev=995389&view=auto
==============================================================================
--- tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/AbstractPOIFSExtractor.java (added)
+++ tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/AbstractPOIFSExtractor.java Thu Sep  9 11:55:32 2010
@@ -0,0 +1,99 @@
+/*
+ * 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.microsoft;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.poi.poifs.filesystem.DirectoryEntry;
+import org.apache.poi.poifs.filesystem.DocumentEntry;
+import org.apache.poi.poifs.filesystem.DocumentInputStream;
+import org.apache.poi.poifs.filesystem.Entry;
+import org.apache.poi.poifs.filesystem.POIFSFileSystem;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.io.TikaInputStream;
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.parser.EmptyParser;
+import org.apache.tika.parser.ParseContext;
+import org.apache.tika.parser.Parser;
+import org.apache.tika.parser.microsoft.OfficeParser.POIFSDocumentType;
+import org.apache.tika.sax.EmbeddedContentHandler;
+import org.apache.tika.sax.XHTMLContentHandler;
+import org.xml.sax.SAXException;
+
+abstract class AbstractPOIFSExtractor {
+
+    private final ParseContext context;
+
+    protected AbstractPOIFSExtractor(ParseContext context) {
+        this.context = context;
+    }
+
+    /**
+     * Handle an office document that's embedded at the POIFS level
+     */
+    protected void handleEmbededOfficeDoc(
+            DirectoryEntry dir, XHTMLContentHandler xhtml)
+            throws IOException, SAXException, TikaException {
+       // Need to dump the directory out to a new temp file, so
+       //  it's stand along
+       POIFSFileSystem newFS = new POIFSFileSystem();
+       copy(dir, newFS.getRoot());
+
+       File tmpFile = File.createTempFile("tika", ".ole2");
+       try {
+           FileOutputStream out = new FileOutputStream(tmpFile);
+           newFS.writeFilesystem(out);
+           out.close();
+
+           // What kind of document is it?
+           Metadata metadata = new Metadata();
+           POIFSDocumentType type = POIFSDocumentType.detectType(dir);
+           metadata.set(Metadata.CONTENT_TYPE, type.getType().toString());
+
+           // Trigger for the document itself 
+           TikaInputStream embeded = TikaInputStream.get(tmpFile);
+           try {
+               Parser parser = context.get(Parser.class, EmptyParser.INSTANCE);
+               parser.parse(
+                       embeded, new EmbeddedContentHandler(xhtml),
+                       metadata, context);
+           } finally {
+               embeded.close();
+           }
+       } finally {
+           tmpFile.delete();
+       }
+    }
+
+    protected void copy(DirectoryEntry sourceDir, DirectoryEntry destDir)
+            throws IOException {
+        for (Entry entry : sourceDir) {
+            if (entry instanceof DirectoryEntry) {
+                // Need to recurse
+                DirectoryEntry newDir = destDir.createDirectory(entry.getName());
+                copy((DirectoryEntry)entry, newDir);
+            } else {
+                // Copy entry
+                InputStream contents = new DocumentInputStream((DocumentEntry)entry); 
+                destDir.createDocument(entry.getName(), contents);
+            }
+        }
+    }
+}

Modified: tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ExcelExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ExcelExtractor.java?rev=995389&r1=995388&r2=995389&view=diff
==============================================================================
--- tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ExcelExtractor.java (original)
+++ tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/ExcelExtractor.java Thu Sep  9 11:55:32 2010
@@ -41,17 +41,21 @@ import org.apache.poi.hssf.record.Extend
 import org.apache.poi.hssf.record.FormatRecord;
 import org.apache.poi.hssf.record.FormulaRecord;
 import org.apache.poi.hssf.record.HyperlinkRecord;
-import org.apache.poi.hssf.record.TextObjectRecord;
 import org.apache.poi.hssf.record.LabelRecord;
 import org.apache.poi.hssf.record.LabelSSTRecord;
 import org.apache.poi.hssf.record.NumberRecord;
 import org.apache.poi.hssf.record.RKRecord;
 import org.apache.poi.hssf.record.Record;
 import org.apache.poi.hssf.record.SSTRecord;
+import org.apache.poi.hssf.record.TextObjectRecord;
 import org.apache.poi.hssf.record.chart.SeriesTextRecord;
 import org.apache.poi.hssf.record.common.UnicodeString;
+import org.apache.poi.poifs.filesystem.DirectoryEntry;
 import org.apache.poi.poifs.filesystem.DocumentInputStream;
+import org.apache.poi.poifs.filesystem.Entry;
 import org.apache.poi.poifs.filesystem.POIFSFileSystem;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.parser.ParseContext;
 import org.apache.tika.sax.XHTMLContentHandler;
 import org.xml.sax.SAXException;
 
@@ -72,7 +76,7 @@ import org.xml.sax.SAXException;
  * @see <a href="http://poi.apache.org/hssf/how-to.html#event_api">
  * POI Event API How To</a>
  */
-public class ExcelExtractor {
+public class ExcelExtractor extends AbstractPOIFSExtractor {
 
     /**
      * <code>true</code> if the HSSFListener should be registered
@@ -82,6 +86,10 @@ public class ExcelExtractor {
      */
     private boolean listenForAllRecords = false;
 
+    public ExcelExtractor(ParseContext context) {
+        super(context);
+    }
+
     /**
      * Returns <code>true</code> if this parser is configured to listen
      * for all records instead of just the specified few.
@@ -120,6 +128,17 @@ public class ExcelExtractor {
         TikaHSSFListener listener = new TikaHSSFListener(xhtml, locale);
         listener.processFile(filesystem, isListenForAllRecords());
         listener.throwStoredException();
+
+        for (Entry entry : filesystem.getRoot()) {
+            if (entry.getName().startsWith("MBD")
+                    && entry instanceof DirectoryEntry) {
+                try {
+                    handleEmbededOfficeDoc((DirectoryEntry) entry, xhtml);
+                } catch (TikaException e) {
+                    // ignore parse errors from embedded documents
+                }
+            }
+         }
     }
 
     // ======================================================================

Modified: tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java?rev=995389&r1=995388&r2=995389&view=diff
==============================================================================
--- tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java (original)
+++ tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java Thu Sep  9 11:55:32 2010
@@ -28,7 +28,6 @@ import java.util.Set;
 import org.apache.poi.hdgf.extractor.VisioTextExtractor;
 import org.apache.poi.hpbf.extractor.PublisherTextExtractor;
 import org.apache.poi.hslf.extractor.PowerPointExtractor;
-import org.apache.poi.hwpf.extractor.WordExtractor;
 import org.apache.poi.poifs.crypt.Decryptor;
 import org.apache.poi.poifs.crypt.EncryptionInfo;
 import org.apache.poi.poifs.filesystem.DirectoryEntry;
@@ -185,27 +184,7 @@ public class OfficeParser implements Par
                     xhtml.element("p", publisherTextExtractor.getText());
                     break;
                 case WORDDOCUMENT:
-                    WordExtractor wordExtractor = new WordExtractor(filesystem);
-
-                    addTextIfAny(xhtml, "header", wordExtractor.getHeaderText());
-
-                    for (String paragraph : wordExtractor.getParagraphText()) {
-                        xhtml.element("p", paragraph);
-                    }
-
-                    for (String paragraph : wordExtractor.getFootnoteText()) {
-                        xhtml.element("p", paragraph);
-                    }
-
-                    for (String paragraph : wordExtractor.getCommentsText()) {
-                        xhtml.element("p", paragraph);
-                    }
-
-                    for (String paragraph : wordExtractor.getEndnoteText()) {
-                        xhtml.element("p", paragraph);
-                    }
-
-                    addTextIfAny(xhtml, "footer", wordExtractor.getFooterText());
+                    new WordExtractor(context).parse(filesystem, xhtml);
                     break;
                 case POWERPOINT:
                     PowerPointExtractor powerPointExtractor =
@@ -214,7 +193,7 @@ public class OfficeParser implements Par
                     break;
                 case WORKBOOK:
                     Locale locale = context.get(Locale.class, Locale.getDefault());
-                    new ExcelExtractor().parse(filesystem, xhtml, locale);
+                    new ExcelExtractor(context).parse(filesystem, xhtml, locale);
                     break;
                 case VISIO:
                     VisioTextExtractor visioTextExtractor =
@@ -269,22 +248,4 @@ public class OfficeParser implements Par
         metadata.set(Metadata.CONTENT_TYPE, type.toString());
     }
 
-    /**
-     * Outputs a section of text if the given text is non-empty.
-     *
-     * @param xhtml XHTML content handler
-     * @param section the class of the &lt;div/&gt; section emitted
-     * @param text text to be emitted, if any
-     * @throws SAXException if an error occurs
-     */
-    private void addTextIfAny(
-            XHTMLContentHandler xhtml, String section, String text)
-            throws SAXException {
-        if (text != null && text.length() > 0) {
-            xhtml.startElement("div", "class", section);
-            xhtml.element("p", text);
-            xhtml.endElement("div");
-        }
-    }
-
 }

Added: tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/WordExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/WordExtractor.java?rev=995389&view=auto
==============================================================================
--- tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/WordExtractor.java (added)
+++ tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/WordExtractor.java Thu Sep  9 11:55:32 2010
@@ -0,0 +1,93 @@
+/*
+ * 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.microsoft;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.poi.poifs.filesystem.DirectoryEntry;
+import org.apache.poi.poifs.filesystem.Entry;
+import org.apache.poi.poifs.filesystem.POIFSFileSystem;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.parser.ParseContext;
+import org.apache.tika.sax.XHTMLContentHandler;
+import org.xml.sax.SAXException;
+
+public class WordExtractor extends AbstractPOIFSExtractor {
+
+    public WordExtractor(ParseContext context) {
+        super(context);
+    }
+
+    protected void parse(
+            POIFSFileSystem filesystem, XHTMLContentHandler xhtml)
+            throws IOException, SAXException, TikaException {
+        org.apache.poi.hwpf.extractor.WordExtractor wordExtractor =
+            new org.apache.poi.hwpf.extractor.WordExtractor(filesystem);
+
+        addTextIfAny(xhtml, "header", wordExtractor.getHeaderText());
+
+        for (String paragraph : wordExtractor.getParagraphText()) {
+            xhtml.element("p", paragraph);
+        }
+
+        for (String paragraph : wordExtractor.getFootnoteText()) {
+            xhtml.element("p", paragraph);
+        }
+
+        for (String paragraph : wordExtractor.getCommentsText()) {
+            xhtml.element("p", paragraph);
+        }
+
+        for (String paragraph : wordExtractor.getEndnoteText()) {
+            xhtml.element("p", paragraph);
+        }
+
+        addTextIfAny(xhtml, "footer", wordExtractor.getFooterText());
+
+        try {
+            DirectoryEntry op =
+                (DirectoryEntry) filesystem.getRoot().getEntry("ObjectPool");
+            for (Entry entry : op) {
+                if (entry.getName().startsWith("_")
+                        && entry instanceof DirectoryEntry) {
+                    handleEmbededOfficeDoc((DirectoryEntry) entry, xhtml);
+                }
+            }
+        } catch(FileNotFoundException e) {
+        }
+    }
+
+    /**
+     * Outputs a section of text if the given text is non-empty.
+     *
+     * @param xhtml XHTML content handler
+     * @param section the class of the &lt;div/&gt; section emitted
+     * @param text text to be emitted, if any
+     * @throws SAXException if an error occurs
+     */
+    private void addTextIfAny(
+            XHTMLContentHandler xhtml, String section, String text)
+            throws SAXException {
+        if (text != null && text.length() > 0) {
+            xhtml.startElement("div", "class", section);
+            xhtml.element("p", text);
+            xhtml.endElement("div");
+        }
+    }
+
+}

Copied: tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIContainerExtractionTest.java (from r995359, tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java)
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIContainerExtractionTest.java?p2=tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIContainerExtractionTest.java&p1=tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java&r1=995359&r2=995389&rev=995389&view=diff
==============================================================================
--- tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java (original)
+++ tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIContainerExtractionTest.java Thu Sep  9 11:55:32 2010
@@ -24,10 +24,15 @@ import junit.framework.TestCase;
 
 import org.apache.tika.extractor.ContainerEmbededResourceHandler;
 import org.apache.tika.extractor.ContainerExtractor;
+import org.apache.tika.extractor.ParserContainerExtractor;
 import org.apache.tika.io.TikaInputStream;
 import org.apache.tika.mime.MediaType;
 
-public class POIFSContainerExtractorTest extends TestCase {
+/**
+ * Tests that the various POI powered parsers are
+ *  able to extract their embeded contents.
+ */
+public class POIContainerExtractionTest extends TestCase {
     private static final MediaType TYPE_DOC = MediaType.application("msword");
     private static final MediaType TYPE_PPT = MediaType.application("vnd.ms-powerpoint");
     private static final MediaType TYPE_XLS = MediaType.application("vnd.ms-excel");
@@ -36,7 +41,7 @@ public class POIFSContainerExtractorTest
      * For office files which don't have anything embeded in them
      */
     public void testWithoutEmbeded() throws Exception {
-       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       ContainerExtractor extractor = new ParserContainerExtractor();
        
        String[] files = new String[] {
              "testEXCEL.xls", "testWORD.doc", "testPPT.ppt",
@@ -62,7 +67,7 @@ public class POIFSContainerExtractorTest
      *  office files in them
      */
     public void testEmbededImages() throws Exception {
-       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       ContainerExtractor extractor = new ParserContainerExtractor();
        TrackingHandler handler;
        
        // Excel with 1 image
@@ -95,7 +100,7 @@ public class POIFSContainerExtractorTest
      *           -> image
      */
     public void testEmbededOfficeFiles() throws Exception {
-       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       ContainerExtractor extractor = new ParserContainerExtractor();
        TrackingHandler handler;
        
        // Excel with a word doc and a powerpoint doc, both of which have images in them
@@ -148,7 +153,7 @@ public class POIFSContainerExtractorTest
     }
     
     private TrackingHandler process(String filename, ContainerExtractor extractor, boolean recurse) throws Exception {
-       InputStream input = POIFSContainerExtractorTest.class.getResourceAsStream(
+       InputStream input = POIContainerExtractionTest.class.getResourceAsStream(
              "/test-documents/" + filename);
         TikaInputStream stream = TikaInputStream.get(input);