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/08 19:09:36 UTC

svn commit: r995157 - in /tika/trunk: tika-core/src/main/java/org/apache/tika/config/ 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/micr...

Author: nick
Date: Wed Sep  8 17:09:36 2010
New Revision: 995157

URL: http://svn.apache.org/viewvc?rev=995157&view=rev
Log:
Initial work on Container Extractors (TIKA-509)
Basics of the interfaces and key classes are included, along with a partial POIFS extractor implementation.

Added:
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerEmbededResourceHandler.java
    tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerExtractor.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-core/src/main/java/org/apache/tika/config/TikaConfig.java
    tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/OfficeParser.java

Modified: tika/trunk/tika-core/src/main/java/org/apache/tika/config/TikaConfig.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-core/src/main/java/org/apache/tika/config/TikaConfig.java?rev=995157&r1=995156&r2=995157&view=diff
==============================================================================
--- tika/trunk/tika-core/src/main/java/org/apache/tika/config/TikaConfig.java (original)
+++ tika/trunk/tika-core/src/main/java/org/apache/tika/config/TikaConfig.java Wed Sep  8 17:09:36 2010
@@ -20,8 +20,10 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 
 import javax.imageio.spi.ServiceRegistry;
@@ -30,6 +32,7 @@ import javax.xml.parsers.DocumentBuilder
 import javax.xml.parsers.ParserConfigurationException;
 
 import org.apache.tika.exception.TikaException;
+import org.apache.tika.extractor.ContainerExtractor;
 import org.apache.tika.mime.MediaType;
 import org.apache.tika.mime.MediaTypeRegistry;
 import org.apache.tika.mime.MimeTypeException;
@@ -50,6 +53,9 @@ public class TikaConfig {
 
     private final Map<MediaType, Parser> parsers =
         new HashMap<MediaType, Parser>();
+    
+    private final List<ContainerExtractor> containerExtractors =
+        new ArrayList<ContainerExtractor>();
 
     private final MimeTypes mimeTypes;
 
@@ -247,6 +253,10 @@ public class TikaConfig {
     public Map<MediaType, Parser> getParsers() {
         return parsers;
     }
+    
+    public List<ContainerExtractor> getContainerExtractors() {
+        return containerExtractors;
+    }
 
     public MimeTypes getMimeRepository(){
         return mimeTypes;

Added: tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java?rev=995157&view=auto
==============================================================================
--- tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java (added)
+++ tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/AutoContainerExtractor.java Wed Sep  8 17:09:36 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.extractor;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.tika.config.TikaConfig;
+import org.apache.tika.detect.Detector;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.io.CountingInputStream;
+import org.apache.tika.io.TikaInputStream;
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.mime.MediaType;
+import org.apache.tika.sax.SecureContentHandler;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+
+public class AutoContainerExtractor implements ContainerExtractor {
+
+    /**
+     * Serial version UID
+     */
+    private static final long serialVersionUID = 2261131045580861514L;
+    
+    private List<ContainerExtractor> extractors;
+
+    /**
+     * Creates an auto-detecting parser instance using the default Tika
+     * configuration.
+     */
+    public AutoContainerExtractor() {
+        this(TikaConfig.getDefaultConfig());
+    }
+
+    public AutoContainerExtractor(TikaConfig config) {
+        this.extractors = config.getContainerExtractors();
+    }
+
+    public List<ContainerExtractor> getExtractors() {
+        return extractors;
+    }
+
+    public void setExtractors(List<ContainerExtractor> extractors) {
+        this.extractors = extractors;
+    }
+    
+    
+    public boolean isSupported(TikaInputStream input) throws IOException {
+        for(ContainerExtractor extractor : extractors) {
+            if(extractor.isSupported(input)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+
+    public void extract(TikaInputStream stream, ContainerExtractor recurseExtractor,
+                    ContainerEmbededResourceHandler handler) 
+             throws IOException, TikaException {
+       // Find a suitable extractor
+       ContainerExtractor extractor = null;
+       for(ContainerExtractor e : extractors) {
+          if(e.isSupported(stream)) {
+             extractor = e;
+             break;
+          }
+       }
+       if(extractor == null) {
+          throw new TikaException("Not a supported container format - no extractor found");
+       }
+       
+       // Have the extractor process it for us
+       extractor.extract(stream, recurseExtractor, handler);
+    }
+}

Added: tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerEmbededResourceHandler.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerEmbededResourceHandler.java?rev=995157&view=auto
==============================================================================
--- tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerEmbededResourceHandler.java (added)
+++ tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerEmbededResourceHandler.java Wed Sep  8 17:09:36 2010
@@ -0,0 +1,44 @@
+/**
+ * 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.InputStream;
+
+import org.apache.tika.mime.MediaType;
+
+/**
+ * Tika container extractor callback interface.
+ * To work with a {@link ContainerExtractor}, your code needs
+ *  to implement this interface.
+ */
+public interface ContainerEmbededResourceHandler {
+    /**
+     * Called to process an embeded resource within the container.
+     * This will be called once per embeded resource within the
+     *  container, along with whatever details are available on
+     *  the embeded resource.
+     *  
+     * TODO Don't pass in the input stream, so that if the entry
+     *  isn't desired then work isn't done to extract it
+     * 
+     * @since Apache Tika 0.8
+     * @param filename The filename of the embeded resource, if known
+     * @param mediaType The media type of the embeded resource, if known
+     * @param stream The contents of the embeded resource
+     */
+    void handle(String filename, MediaType mediaType, InputStream stream);
+}

Added: tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerExtractor.java?rev=995157&view=auto
==============================================================================
--- tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerExtractor.java (added)
+++ tika/trunk/tika-core/src/main/java/org/apache/tika/extractor/ContainerExtractor.java Wed Sep  8 17:09:36 2010
@@ -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.extractor;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Set;
+
+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.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+
+/**
+ * Tika container extractor interface.
+ * Container Extractors provide access to the embeded
+ *  resources within container formats such as .zip and .doc 
+ */
+public interface ContainerExtractor extends Serializable {
+    /**
+     * Is this Container Extractor able to process the
+     *  supplied container?
+     * @since Apache Tika 0.8
+     */
+    boolean isSupported(TikaInputStream input) throws IOException;
+
+    /**
+     * Processes a container file, and extracts all the embeded
+     * resources from within it.
+     * <p>
+     * The {@link ContainerEmbededResourceHandler} you supply will
+     * be called for each embeded resource in the container. It is
+     * up to you whether you process the contents of the resource or not. 
+     * <p>
+     * The given document stream is consumed but not closed by this method.
+     * The responsibility to close the stream remains on the caller.
+     * <p>
+     * If required, nested containers (such as a .docx within a .zip)
+     * can automatically be recursed into, and processed inline. If
+     * no recurseExtractor is given, the nested containers will be
+     * treated as with any other embeded resources.
+     *
+     * @since Apache Tika 0.8
+     * @param stream the document stream (input)
+     * @param recurseExtractor the extractor to use on any embeded containers 
+     * @param handler handler for the embeded files (output)
+     * @throws IOException if the document stream could not be read
+     * @throws TikaException if the container could not be parsed
+     */
+    void extract(
+            TikaInputStream stream, ContainerExtractor recurseExtractor,
+            ContainerEmbededResourceHandler handler)
+            throws IOException, TikaException;
+}

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=995157&r1=995156&r2=995157&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 Wed Sep  8 17:09:36 2010
@@ -31,7 +31,7 @@ import org.apache.poi.hslf.extractor.Pow
 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.DirectoryNode;
+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;
@@ -51,7 +51,9 @@ import org.xml.sax.SAXException;
  * Defines a Microsoft document content extractor.
  */
 public class OfficeParser implements Parser {
-    private static final Set<MediaType> SUPPORTED_TYPES =
+   private static final long serialVersionUID = 7393462244028653479L;
+   
+   private static final Set<MediaType> SUPPORTED_TYPES =
         Collections.unmodifiableSet(new HashSet<MediaType>(Arrays.asList(
         	POIFSDocumentType.WORKBOOK.type,
         	POIFSDocumentType.OLE10_NATIVE.type,
@@ -97,7 +99,7 @@ public class OfficeParser implements Par
             return detectType(fs.getRoot());
         }
 
-        public static POIFSDocumentType detectType(DirectoryNode node) {
+        public static POIFSDocumentType detectType(DirectoryEntry node) {
             for (Entry entry : node) {
                 POIFSDocumentType type = detectType(entry);
                 if (type!=UNKNOWN) {

Added: tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/POIFSContainerExtractor.java
URL: http://svn.apache.org/viewvc/tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/POIFSContainerExtractor.java?rev=995157&view=auto
==============================================================================
--- tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/POIFSContainerExtractor.java (added)
+++ tika/trunk/tika-parsers/src/main/java/org/apache/tika/parser/microsoft/POIFSContainerExtractor.java Wed Sep  8 17:09:36 2010
@@ -0,0 +1,179 @@
+/*
+ * 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.FileNotFoundException;
+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.DirectoryNode;
+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.poi.poifs.storage.HeaderBlockConstants;
+import org.apache.poi.util.IOUtils;
+import org.apache.poi.util.LittleEndian;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.extractor.AutoContainerExtractor;
+import org.apache.tika.extractor.ContainerEmbededResourceHandler;
+import org.apache.tika.extractor.ContainerExtractor;
+import org.apache.tika.io.TikaInputStream;
+import org.apache.tika.parser.microsoft.OfficeParser.POIFSDocumentType;
+
+/**
+ * A Container Extractor that handles extracting resources embeded
+ *  in Microsoft office files, eg images and excel files embeded
+ *  in a word document.
+ */
+public class POIFSContainerExtractor implements ContainerExtractor {
+   private static final long serialVersionUID = 223856361982352348L;
+   
+   public boolean isSupported(TikaInputStream input) throws IOException {
+      // Grab the first 8 bytes, used to do container detection
+      input.mark(8);
+      byte[] first8 = new byte[8];
+      IOUtils.readFully(input, first8);
+      input.reset();
+      
+      // Is it one of ours?
+      long ole2Signature = LittleEndian.getLong(first8, 0);
+      if(ole2Signature == HeaderBlockConstants._signature) {
+         return true;
+      }
+      return false;
+   }
+
+   public void extract(TikaInputStream stream,
+         ContainerExtractor recurseExtractor,
+         ContainerEmbededResourceHandler handler) throws IOException,
+         TikaException {
+      POIFSFileSystem fs = new POIFSFileSystem(stream);
+      extract(fs.getRoot(), recurseExtractor, handler);
+   }
+      
+   public void extract(DirectoryEntry dir,
+         ContainerExtractor recurseExtractor,
+         ContainerEmbededResourceHandler handler) throws IOException,
+         TikaException {
+      // What kind of thing is it?
+      POIFSDocumentType type = POIFSDocumentType.detectType(dir);
+      switch(type) {
+         case WORKBOOK:
+            // Firstly do any embeded office documents
+            for(Entry entry : dir) {
+               if(entry.getName().startsWith("MBD")) {
+                  handleEmbededOfficeDoc((DirectoryEntry)entry, recurseExtractor, handler);
+               }
+            }
+            
+            // Now do the embeded images
+            // TODO
+            break;
+         case WORDDOCUMENT:
+            // Firstly do any embeded office documents
+            try {
+               DirectoryEntry op = (DirectoryEntry)dir.getEntry("ObjectPool");
+               for(Entry entry : op) {
+                  if(entry.getName().startsWith("_")) {
+                     handleEmbededOfficeDoc((DirectoryEntry)entry, recurseExtractor, handler);
+                  }
+               }
+            } catch(FileNotFoundException e) {}
+            
+            // Now do the embeded images
+            // TODO
+            break;
+         case POWERPOINT:
+            // TODO
+            break;
+         case VISIO:
+            // TODO
+            break;
+         case OUTLOOK:
+            // Firstly do any embeded emails
+            
+            // Now any embeded files
+            break;
+      }
+   }
+
+   /**
+    * Handle an office document that's embeded at the POIFS level
+    */
+   protected void handleEmbededOfficeDoc(DirectoryEntry dir,
+         ContainerExtractor recurseExtractor,
+         ContainerEmbededResourceHandler handler) throws IOException,
+         TikaException {
+      // Is it an embeded ooxml file?
+      try {
+         Entry ooxml = dir.getEntry("Package");
+         // TODO
+      } catch(FileNotFoundException e) {}
+      
+      // Looks to be an embeded OLE2 office file
+      
+      // 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");
+      FileOutputStream out = new FileOutputStream(tmpFile);
+      newFS.writeFilesystem(out);
+      out.close();
+      
+      // What kind of document is it?
+      POIFSDocumentType type = POIFSDocumentType.detectType(dir);
+      
+      // Trigger for the document itself 
+      TikaInputStream embeded = TikaInputStream.get(tmpFile);
+      handler.handle(null, type.getType(), embeded);
+      
+      // If we are recursing, process the document's contents too
+      if(recurseExtractor != null) {
+         if(recurseExtractor instanceof POIFSContainerExtractor ||
+            recurseExtractor instanceof AutoContainerExtractor) {
+            // Shortcut - use already open poifs
+            extract(dir, recurseExtractor, handler);
+         } else {
+            // Long way round, need to use the temporary document
+            recurseExtractor.extract(embeded, recurseExtractor, handler);
+         }
+      }
+      
+      // Tidy up
+      embeded.close();
+      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);
+         }
+      }
+   }
+}

Added: 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/POIFSContainerExtractorTest.java?rev=995157&view=auto
==============================================================================
--- tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java (added)
+++ tika/trunk/tika-parsers/src/test/java/org/apache/tika/parser/microsoft/POIFSContainerExtractorTest.java Wed Sep  8 17:09:36 2010
@@ -0,0 +1,175 @@
+/**
+ * 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.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.tika.extractor.ContainerEmbededResourceHandler;
+import org.apache.tika.extractor.ContainerExtractor;
+import org.apache.tika.io.TikaInputStream;
+import org.apache.tika.mime.MediaType;
+
+public class POIFSContainerExtractorTest 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");
+   
+    /**
+     * For office files which don't have anything embeded in them
+     */
+    public void testWithoutEmbeded() throws Exception {
+       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       
+       String[] files = new String[] {
+             "testEXCEL.xls", "testWORD.doc", "testPPT.ppt",
+             "testVISIO.vsd", "test-outlook.msg"
+       };
+       for(String file : files) {
+          // Process it without recursing
+          TrackingHandler handler = process(file, extractor, false);
+          
+          // Won't have fired
+          assertEquals(0, handler.filenames.size());
+          assertEquals(0, handler.mediaTypes.size());
+          
+          // Ditto with recursing
+          handler = process(file, extractor, true);
+          assertEquals(0, handler.filenames.size());
+          assertEquals(0, handler.mediaTypes.size());
+       }
+    }
+    
+    /**
+     * Office files with embeded images, but no other
+     *  office files in them
+     */
+    public void testEmbededImages() throws Exception {
+       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       TrackingHandler handler;
+       
+       // Excel with 1 image
+       handler = process("testEXCEL_1img.xls", extractor, false);
+       // TODO
+       assertEquals(0, handler.filenames.size());
+       assertEquals(0, handler.mediaTypes.size());
+       
+       // PowerPoint with 2 images + sound
+       // TODO
+       
+       // Word with 1 image
+       // TODO
+       
+       // Word with 3 images
+       // TODO
+    }
+    
+    /**
+     * Office files which have other office files
+     *  embeded into them. The embeded office files
+     *  will sometimes have images in them.
+     *  
+     *  eg xls
+     *       -> word
+     *           -> image
+     *           -> image
+     *       -> powerpoint
+     *       -> excel
+     *           -> image
+     */
+    public void testEmbededOfficeFiles() throws Exception {
+       POIFSContainerExtractor extractor = new POIFSContainerExtractor();
+       TrackingHandler handler;
+       
+       // Excel with a word doc and a powerpoint doc, both of which have images in them
+       // Without recursion, should see both
+       handler = process("testEXCEL_embeded.xls", extractor, false);
+       assertEquals(2, handler.filenames.size());
+       assertEquals(2, handler.mediaTypes.size());
+       
+       // We don't know their filenames
+       assertEquals(null, handler.filenames.get(0));
+       assertEquals(null, handler.filenames.get(1));
+       // But we do know their types
+       assertEquals(TYPE_PPT, handler.mediaTypes.get(0));
+       assertEquals(TYPE_DOC, handler.mediaTypes.get(1));
+       
+       // With recursion, should get their images too
+       handler = process("testEXCEL_embeded.xls", extractor, true);
+       // TODO
+       
+       
+       // Word with .docx, powerpoint and excel
+       handler = process("testWORD_embeded.doc", extractor, false);
+       assertEquals(3, handler.filenames.size());
+       assertEquals(3, handler.mediaTypes.size());
+       
+       // We don't know their filenames
+       assertEquals(null, handler.filenames.get(0));
+       assertEquals(null, handler.filenames.get(1));
+       assertEquals(null, handler.filenames.get(2));
+       // But we do know their types
+       assertEquals(MediaType.application("x-tika-msoffice"), handler.mediaTypes.get(0)); // TODO
+       assertEquals(TYPE_PPT, handler.mediaTypes.get(1));
+       assertEquals(TYPE_XLS, handler.mediaTypes.get(2));
+       
+       // With recursion, should get their images too
+       handler = process("testWORD_embeded.doc", extractor, true);
+       // TODO
+       
+       
+       // PowerPoint with excel and word
+       // TODO
+       
+       
+       // Outlook with a text file and a word document
+       // TODO
+       
+       
+       // Outlook with a pdf and another outlook message
+       // TODO
+    }
+    
+    private TrackingHandler process(String filename, ContainerExtractor extractor, boolean recurse) throws Exception {
+       InputStream input = POIFSContainerExtractorTest.class.getResourceAsStream(
+             "/test-documents/" + filename);
+        TikaInputStream stream = TikaInputStream.get(input);
+        
+        assertEquals(true, extractor.isSupported(stream));
+        
+        // Process it
+        TrackingHandler handler = new TrackingHandler();
+        extractor.extract(stream, null, handler);
+        
+        // So they can check what happened
+        return handler;
+    }
+    
+    private static class TrackingHandler implements ContainerEmbededResourceHandler {
+       private List<String> filenames = new ArrayList<String>();
+       private List<MediaType> mediaTypes = new ArrayList<MediaType>();
+       
+       public void handle(String filename, MediaType mediaType,
+            InputStream stream) {
+          filenames.add(filename);
+          mediaTypes.add(mediaType);
+      }
+    }
+}