You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tika.apache.org by ta...@apache.org on 2016/06/27 14:22:53 UTC

tika git commit: TIKA-2022 -- add parser for applefile

Repository: tika
Updated Branches:
  refs/heads/2.x cd12917fa -> b14b47e76


TIKA-2022 -- add parser for applefile


Project: http://git-wip-us.apache.org/repos/asf/tika/repo
Commit: http://git-wip-us.apache.org/repos/asf/tika/commit/b14b47e7
Tree: http://git-wip-us.apache.org/repos/asf/tika/tree/b14b47e7
Diff: http://git-wip-us.apache.org/repos/asf/tika/diff/b14b47e7

Branch: refs/heads/2.x
Commit: b14b47e76a4cba829b17d5180ebd591e641ad683
Parents: cd12917
Author: tballison <ta...@mitre.org>
Authored: Mon Jun 27 10:22:43 2016 -0400
Committer: tballison <ta...@mitre.org>
Committed: Mon Jun 27 10:22:43 2016 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../org/apache/tika/mime/tika-mimetypes.xml     |  13 +-
 .../org/apache/tika/module/office/BundleIT.java |  24 +--
 .../parser/apple/AppleSingleFileParser.java     | 205 +++++++++++++++++++
 .../services/org.apache.tika.parser.Parser      |   2 +-
 .../parser/apple/AppleSingleFileParserTest.java |  40 ++++
 .../test-documents/testAppleSingleFile.pdf      | Bin 0 -> 54926 bytes
 7 files changed, 271 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6a6bca1..766780f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,8 @@ Release 2.0 - ???
 
 Release 1.14 - ???
 
+  * Add parser for applefile (AppleSingle) (TIKA-2022)
+
   * Add mime types, mime magic and/or globs for:
      * Endnote Import File (TIKA-2011)
      * DJVU files (TIKA-2009)

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-core/src/main/resources/org/apache/tika/mime/tika-mimetypes.xml
----------------------------------------------------------------------
diff --git a/tika-core/src/main/resources/org/apache/tika/mime/tika-mimetypes.xml b/tika-core/src/main/resources/org/apache/tika/mime/tika-mimetypes.xml
index 83cd239..e07f449 100644
--- a/tika-core/src/main/resources/org/apache/tika/mime/tika-mimetypes.xml
+++ b/tika-core/src/main/resources/org/apache/tika/mime/tika-mimetypes.xml
@@ -42,7 +42,12 @@
   <mime-type type="application/andrew-inset">
     <glob pattern="*.ez"/>
   </mime-type>
-  <mime-type type="application/applefile"/>
+
+  <mime-type type="application/applefile">
+    <magic priority="50">
+      <match value="0x00051600" type="string" offset="0"/>
+    </magic>
+  </mime-type>
   <mime-type type="application/applixware">
     <glob pattern="*.aw"/>
   </mime-type>
@@ -5282,7 +5287,11 @@
   </mime-type>
 
   <mime-type type="multipart/alternative"/>
-  <mime-type type="multipart/appledouble"/>
+  <mime-type type="multipart/appledouble">
+    <magic priority="50">
+      <match value="0x00051607" type="string" offset="0"/>
+    </magic>
+  </mime-type>
   <mime-type type="multipart/byteranges"/>
   <mime-type type="multipart/digest"/>
   <mime-type type="multipart/encrypted"/>

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-parser-bundles/tika-parser-office-bundle/src/test/java/org/apache/tika/module/office/BundleIT.java
----------------------------------------------------------------------
diff --git a/tika-parser-bundles/tika-parser-office-bundle/src/test/java/org/apache/tika/module/office/BundleIT.java b/tika-parser-bundles/tika-parser-office-bundle/src/test/java/org/apache/tika/module/office/BundleIT.java
index 6336ddf..cd9c66e 100644
--- a/tika-parser-bundles/tika-parser-office-bundle/src/test/java/org/apache/tika/module/office/BundleIT.java
+++ b/tika-parser-bundles/tika-parser-office-bundle/src/test/java/org/apache/tika/module/office/BundleIT.java
@@ -16,6 +16,17 @@
  */
 package org.apache.tika.module.office;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.ops4j.pax.exam.CoreOptions.bundle;
+import static org.ops4j.pax.exam.CoreOptions.junitBundles;
+import static org.ops4j.pax.exam.CoreOptions.options;
+
+import javax.inject.Inject;
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+
 import org.apache.tika.parser.Parser;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -28,17 +39,6 @@ import org.osgi.framework.Bundle;
 import org.osgi.framework.BundleContext;
 import org.osgi.framework.ServiceReference;
 
-import javax.inject.Inject;
-import java.io.File;
-import java.io.IOException;
-import java.net.URISyntaxException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.ops4j.pax.exam.CoreOptions.bundle;
-import static org.ops4j.pax.exam.CoreOptions.junitBundles;
-import static org.ops4j.pax.exam.CoreOptions.options;
-
 
 @RunWith(PaxExam.class)
 @ExamReactorStrategy(PerMethod.class)
@@ -80,6 +80,6 @@ public class BundleIT {
     @Test
     public void testServicesCreated() throws Exception {
         ServiceReference[] services = bc.getAllServiceReferences(Parser.class.getName(), null);
-        assertEquals("Not all Services have started", 25, services.length);
+        assertEquals("Not all Services have started", 26, services.length);
     }
 }

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-parser-modules/tika-parser-office-module/src/main/java/org/apache/tika/parser/apple/AppleSingleFileParser.java
----------------------------------------------------------------------
diff --git a/tika-parser-modules/tika-parser-office-module/src/main/java/org/apache/tika/parser/apple/AppleSingleFileParser.java b/tika-parser-modules/tika-parser-office-module/src/main/java/org/apache/tika/parser/apple/AppleSingleFileParser.java
new file mode 100644
index 0000000..ffb5759
--- /dev/null
+++ b/tika-parser-modules/tika-parser-office-module/src/main/java/org/apache/tika/parser/apple/AppleSingleFileParser.java
@@ -0,0 +1,205 @@
+/*
+ * 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.apple;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.input.CloseShieldInputStream;
+import org.apache.tika.exception.TikaException;
+import org.apache.tika.extractor.EmbeddedDocumentExtractor;
+import org.apache.tika.extractor.ParsingEmbeddedDocumentExtractor;
+import org.apache.tika.io.EndianUtils;
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.mime.MediaType;
+import org.apache.tika.parser.AbstractParser;
+import org.apache.tika.parser.ParseContext;
+import org.apache.tika.sax.XHTMLContentHandler;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+
+/**
+ * Parser that strips the header off of AppleSingle and AppleDouble
+ * files.
+ * <p>
+ * See <a href="http://kaiser-edv.de/documents/AppleSingle_AppleDouble.pdf">spec document</a>.
+ */
+public class AppleSingleFileParser extends AbstractParser {
+
+    /**
+     * Entry types
+     */
+    public static final int DATA_FORK = 1;
+    public static final int RESOURCE_FORK = 2;
+    public static final int REAL_NAME = 3;
+    public static final int COMMENT = 4;
+    public static final int ICON_BW = 5;
+    public static final int ICON_COLOR = 6;
+    //7?!
+    public static final int FILE_DATES_INFO = 8;
+    public static final int FINDER_INFO = 9;
+    public static final int MACINTOSH_FILE_INFO = 10;
+    public static final int PRODOS_FILE_INFO = 11;
+    public static final int MSDOS_FILE_INFO = 12;
+    public static final int SHORT_NAME = 13;
+    public static final int AFP_FILE_INFO = 14;
+    public static final int DIRECTORY_ID = 15;
+
+    private static final Set<MediaType> SUPPORTED_TYPES =
+            Collections.singleton(MediaType.application("applefile"));
+
+    public Set<MediaType> getSupportedTypes(ParseContext context) {
+        return SUPPORTED_TYPES;
+    }
+
+    @Override
+    public void parse(InputStream stream, ContentHandler handler,
+                      Metadata metadata, ParseContext context)
+            throws IOException, SAXException, TikaException {
+
+        EmbeddedDocumentExtractor ex = context.get(EmbeddedDocumentExtractor.class);
+
+        if (ex == null) {
+            ex = new ParsingEmbeddedDocumentExtractor(context);
+        }
+
+        short numEntries = readThroughNumEntries(stream);
+        long bytesRead = 26;
+        List<FieldInfo> fieldInfoList = getSortedFieldInfoList(stream, numEntries);
+        bytesRead += 12*numEntries;
+        Metadata embeddedMetadata = new Metadata();
+        bytesRead = processFieldEntries(stream, fieldInfoList, embeddedMetadata, bytesRead);
+        FieldInfo contentFieldInfo = getContentFieldInfo(fieldInfoList);
+        XHTMLContentHandler xhtml = new XHTMLContentHandler(handler, metadata);
+        xhtml.startDocument();
+        if (contentFieldInfo != null) {
+            System.out.println(contentFieldInfo.offset + " "+bytesRead);
+            long diff = contentFieldInfo.offset-bytesRead;
+            IOUtils.skipFully(stream, diff);
+            if (ex.shouldParseEmbedded(embeddedMetadata)) {
+                // TODO: we should probably add a readlimiting wrapper around this
+                // stream to ensure that not more than contentFieldInfo.length bytes
+                // are read
+                ex.parseEmbedded(new CloseShieldInputStream(stream),
+                        xhtml, embeddedMetadata, false);
+            }
+        }
+        xhtml.endDocument();
+
+    }
+
+    private FieldInfo getContentFieldInfo(List<FieldInfo> fieldInfoList) {
+        for (FieldInfo fieldInfo : fieldInfoList) {
+            if (fieldInfo.entryId == 1) {
+                return fieldInfo;
+            }
+        }
+        return null;
+    }
+
+    private long processFieldEntries(InputStream stream, List<FieldInfo> fieldInfoList,
+                                     Metadata embeddedMetadata, long bytesRead) throws IOException, TikaException {
+        byte[] buffer = null;
+        for (FieldInfo f : fieldInfoList) {
+            long diff = f.offset - bytesRead;
+            //just in case
+            IOUtils.skipFully(stream, diff);
+            bytesRead += diff;
+            if (f.entryId == REAL_NAME) {
+                if (f.length > Integer.MAX_VALUE) {
+                    throw new TikaException("File name length can't be > integer max");
+                }
+                buffer = new byte[(int)f.length];
+                IOUtils.readFully(stream, buffer);
+                bytesRead += f.length;
+                String originalFileName = new String(buffer, 0, buffer.length, StandardCharsets.US_ASCII);
+                //TODO: figure out correct metadata key
+                //embeddedMetadata.set(TikaCoreProperties.IDENTIFIER, originalFileName);
+            } else if (f.entryId != DATA_FORK) {
+                IOUtils.skipFully(stream, f.length);
+                bytesRead += f.length;
+            }
+        }
+        return bytesRead;
+    }
+
+
+    private List<FieldInfo> getSortedFieldInfoList(InputStream stream, short numEntries) throws IOException, TikaException {
+        //this is probably overkill.  I'd hope that these were already
+        //in order.  This ensures it.
+        List<FieldInfo> fieldInfoList = new ArrayList<>(numEntries);
+        for (int i = 0; i < numEntries; i++) {
+            //convert 32-bit unsigned ints to longs
+            fieldInfoList.add(
+                    new FieldInfo(
+                            EndianUtils.readIntBE(stream) & 0x00000000ffffffffL, //entry id
+                            EndianUtils.readIntBE(stream) & 0x00000000ffffffffL, //offset
+                            EndianUtils.readIntBE(stream) & 0x00000000ffffffffL  //length
+                    )
+            );
+        }
+        if (fieldInfoList.size() == 0) {
+            throw new TikaException("AppleSingleFile missing field info");
+        }
+        //make absolutely sure these are in order!
+        Collections.sort(fieldInfoList, new FieldInfoComparator());
+        return fieldInfoList;
+    }
+
+    //read through header until you hit the number of entries
+    private short readThroughNumEntries(InputStream stream) throws TikaException, IOException {
+        //mime
+        EndianUtils.readIntBE(stream);
+        //version
+        long version = EndianUtils.readIntBE(stream);
+        if (version != 0x00020000) {
+            throw new TikaException("Version should have been 0x00020000, but was:"+version);
+        }
+        IOUtils.skipFully(stream, 16);//filler
+        return EndianUtils.readShortBE(stream);//number of entries
+    }
+
+    private class FieldInfo {
+
+        private final long entryId;
+        private final long offset;
+        private final long length;
+
+        private FieldInfo(long entryId, long offset, long length) {
+            this.entryId = entryId;
+            this.offset = offset;
+            this.length = length;
+        }
+    }
+
+    private static class FieldInfoComparator implements Comparator<FieldInfo> {
+
+        @Override
+        public int compare(FieldInfo o1, FieldInfo o2) {
+            return (o1.offset > o2.offset) ? 1 :
+                    (o1.offset == o2.offset) ? 0 : -1 ;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-parser-modules/tika-parser-office-module/src/main/resources/META-INF/services/org.apache.tika.parser.Parser
----------------------------------------------------------------------
diff --git a/tika-parser-modules/tika-parser-office-module/src/main/resources/META-INF/services/org.apache.tika.parser.Parser b/tika-parser-modules/tika-parser-office-module/src/main/resources/META-INF/services/org.apache.tika.parser.Parser
index 1c8cee1..8554f17 100644
--- a/tika-parser-modules/tika-parser-office-module/src/main/resources/META-INF/services/org.apache.tika.parser.Parser
+++ b/tika-parser-modules/tika-parser-office-module/src/main/resources/META-INF/services/org.apache.tika.parser.Parser
@@ -13,7 +13,7 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-
+org.apache.tika.parser.apple.AppleSingleFileParser
 org.apache.tika.parser.chm.ChmParser
 org.apache.tika.parser.microsoft.JackcessParser
 org.apache.tika.parser.microsoft.OfficeParser

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-parser-modules/tika-parser-office-module/src/test/java/org/apache/tika/parser/apple/AppleSingleFileParserTest.java
----------------------------------------------------------------------
diff --git a/tika-parser-modules/tika-parser-office-module/src/test/java/org/apache/tika/parser/apple/AppleSingleFileParserTest.java b/tika-parser-modules/tika-parser-office-module/src/test/java/org/apache/tika/parser/apple/AppleSingleFileParserTest.java
new file mode 100644
index 0000000..c80c94a
--- /dev/null
+++ b/tika-parser-modules/tika-parser-office-module/src/test/java/org/apache/tika/parser/apple/AppleSingleFileParserTest.java
@@ -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.apple;
+
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.tika.TikaTest;
+import org.apache.tika.metadata.Metadata;
+import org.junit.Test;
+
+public class AppleSingleFileParserTest extends TikaTest {
+
+    @Test
+    public void testBasic() throws Exception {
+        List<Metadata> list = getRecursiveJson("testAppleSingleFile.pdf");
+        assertEquals(2, list.size());
+        assertContains(AppleSingleFileParser.class.getName(),
+                Arrays.asList(list.get(0).getValues("X-Parsed-By")));
+        assertEquals("application/pdf", list.get(1).get(Metadata.CONTENT_TYPE));
+    }
+}

http://git-wip-us.apache.org/repos/asf/tika/blob/b14b47e7/tika-test-resources/src/test/resources/test-documents/testAppleSingleFile.pdf
----------------------------------------------------------------------
diff --git a/tika-test-resources/src/test/resources/test-documents/testAppleSingleFile.pdf b/tika-test-resources/src/test/resources/test-documents/testAppleSingleFile.pdf
new file mode 100644
index 0000000..a385313
Binary files /dev/null and b/tika-test-resources/src/test/resources/test-documents/testAppleSingleFile.pdf differ