You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by ch...@apache.org on 2017/01/06 18:08:30 UTC

svn commit: r1777676 - in /jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene: ./ binary/

Author: chetanm
Date: Fri Jan  6 18:08:30 2017
New Revision: 1777676

URL: http://svn.apache.org/viewvc?rev=1777676&view=rev
Log:
OAK-5415 - Refactor Binary text extraction logic from LuceneIndexEditor

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ExtractedTextCache.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ExtractedTextCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ExtractedTextCache.java?rev=1777676&r1=1777675&r2=1777676&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ExtractedTextCache.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ExtractedTextCache.java Fri Jan  6 18:08:30 2017
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 
-class ExtractedTextCache {
+public class ExtractedTextCache {
     private static final String EMPTY_STRING = "";
     private static final Logger log = LoggerFactory.getLogger(ExtractedTextCache.class);
     private volatile PreExtractedTextProvider extractedTextProvider;

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java?rev=1777676&r1=1777675&r2=1777676&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java Fri Jan  6 18:08:30 2017
@@ -27,18 +27,12 @@ import java.util.concurrent.atomic.Atomi
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.google.common.io.CountingInputStream;
-import org.apache.jackrabbit.JcrConstants;
-import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.PathUtils;
-import org.apache.jackrabbit.oak.commons.io.LazyInputStream;
 import org.apache.jackrabbit.oak.plugins.index.IndexEditor;
 import org.apache.jackrabbit.oak.plugins.index.PathFilter;
-import org.apache.jackrabbit.oak.plugins.index.fulltext.ExtractedText;
-import org.apache.jackrabbit.oak.plugins.index.fulltext.ExtractedText.ExtractionResult;
 import org.apache.jackrabbit.oak.plugins.index.lucene.Aggregate.Matcher;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.FunctionIndexProcessor;
 import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriter;
@@ -46,7 +40,6 @@ import org.apache.jackrabbit.oak.plugins
 import org.apache.jackrabbit.oak.plugins.memory.StringPropertyState;
 import org.apache.jackrabbit.oak.spi.commit.Editor;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
-import org.apache.jackrabbit.oak.util.BlobByteSource;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.DoubleField;
@@ -57,13 +50,9 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
 import org.apache.lucene.util.BytesRef;
-import org.apache.tika.metadata.Metadata;
-import org.apache.tika.parser.ParseContext;
-import org.apache.tika.sax.WriteOutContentHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.jackrabbit.JcrConstants.JCR_DATA;
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 import static org.apache.jackrabbit.oak.commons.PathUtils.getName;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.*;
@@ -80,9 +69,7 @@ public class LuceneIndexEditor implement
     private static final Logger log =
             LoggerFactory.getLogger(LuceneIndexEditor.class);
 
-    private static final long SMALL_BINARY = Long.getLong("oak.lucene.smallBinary", 16 * 1024);
-
-    static final String TEXT_EXTRACTION_ERROR = "TextExtractionError";
+    public static final String TEXT_EXTRACTION_ERROR = "TextExtractionError";
 
     private final LuceneIndexEditorContext context;
 
@@ -586,40 +573,12 @@ public class LuceneIndexEditor implement
 
     private List<Field> newBinary(
             PropertyState property, NodeState state, String nodePath, String path) {
-        List<Field> fields = new ArrayList<Field>();
-        Metadata metadata = new Metadata();
-
-        //jcr:mimeType is mandatory for a binary to be indexed
-        String type = state.getString(JcrConstants.JCR_MIMETYPE);
-
-        if (type == null || !isSupportedMediaType(type)) {
-            log.trace(
-                    "[{}] Ignoring binary content for node {} due to unsupported (or null) jcr:mimeType [{}]",
-                    getIndexName(), path, type);
-            return fields;
-        }
-
-        metadata.set(Metadata.CONTENT_TYPE, type);
-        if (JCR_DATA.equals(property.getName())) {
-            String encoding = state.getString(JcrConstants.JCR_ENCODING);
-            if (encoding != null) { // not mandatory
-                metadata.set(Metadata.CONTENT_ENCODING, encoding);
-            }
+        if (!context.isAsyncIndexing()){
+            //Skip text extraction for sync indexing
+            return Collections.emptyList();
         }
 
-        for (Blob v : property.getValue(Type.BINARIES)) {
-            String value = parseStringValue(v, metadata, path, property.getName());
-            if (value == null){
-                continue;
-            }
-
-            if (nodePath != null){
-                fields.add(newFulltextField(nodePath, value, true));
-            } else {
-                fields.add(newFulltextField(value, true));
-            }
-        }
-        return fields;
+        return context.getTextExtractor().newBinary(property, state, nodePath, path);
     }
 
     private boolean augmentCustomFields(final String path, final List<Field> fields,
@@ -941,81 +900,6 @@ public class LuceneIndexEditor implement
         return context.getDefinition().getPathFilter().filter(concat(getPath(), childNodeName));
     }
 
-    private boolean isSupportedMediaType(String type) {
-        return context.isSupportedMediaType(type);
-    }
-
-    private String parseStringValue(Blob v, Metadata metadata, String path, String propertyName) {
-        if (!context.isAsyncIndexing()){
-            //Skip text extraction for sync indexing
-            return null;
-        }
-        String text = context.getExtractedTextCache().get(path, propertyName, v, context.isReindex());
-        if (text == null){
-            text = parseStringValue0(v, metadata, path);
-        }
-        return text;
-    }
-
-    private String parseStringValue0(Blob v, Metadata metadata, String path) {
-        WriteOutContentHandler handler = new WriteOutContentHandler(context.getDefinition().getMaxExtractLength());
-        long start = System.currentTimeMillis();
-        long bytesRead = 0;
-        long length = v.length();
-        if (log.isDebugEnabled()) {
-            log.debug("Extracting {}, {} bytes, id {}", path, length, v.getContentIdentity());
-        }
-        String oldThreadName = null;
-        if (length > SMALL_BINARY) {
-            Thread t = Thread.currentThread();
-            oldThreadName = t.getName();
-            t.setName(oldThreadName + ": Extracting " + path + ", " + length + " bytes");
-        }
-        try {
-            CountingInputStream stream = new CountingInputStream(new LazyInputStream(new BlobByteSource(v)));
-            try {
-                context.getParser().parse(stream, handler, metadata, new ParseContext());
-            } finally {
-                bytesRead = stream.getCount();
-                stream.close();
-            }
-        } catch (LinkageError e) {
-            // Capture and ignore errors caused by extraction libraries
-            // not being present. This is equivalent to disabling
-            // selected media types in configuration, so we can simply
-            // ignore these errors.
-        } catch (Throwable t) {
-            // Capture and report any other full text extraction problems.
-            // The special STOP exception is used for normal termination.
-            if (!handler.isWriteLimitReached(t)) {
-                log.debug(
-                        "[{}] Failed to extract text from a binary property: {}."
-                        + " This is a fairly common case, and nothing to"
-                        + " worry about. The stack trace is included to"
-                        + " help improve the text extraction feature.",
-                        getIndexName(), path, t);
-                context.getExtractedTextCache().put(v, ExtractedText.ERROR);
-                return TEXT_EXTRACTION_ERROR;
-            }
-        } finally {
-            if (oldThreadName != null) {
-                Thread.currentThread().setName(oldThreadName);
-            }
-        }
-        String result = handler.toString();
-        if (bytesRead > 0) {
-            long time = System.currentTimeMillis() - start;
-            int len = result.length();
-            context.recordTextExtractionStats(time, bytesRead, len);
-            if (log.isDebugEnabled()) {
-                log.debug("Extracting {} took {} ms, {} bytes read, {} text size", 
-                        path, time, bytesRead, len);
-            }
-        }
-        context.getExtractedTextCache().put(v,  new ExtractedText(ExtractionResult.SUCCESS, result));
-        return result;
-    }
-
     private String getIndexName() {
         return context.getDefinition().getIndexName();
     }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java?rev=1777676&r1=1777675&r2=1777676&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorContext.java Fri Jan  6 18:08:30 2017
@@ -17,19 +17,15 @@
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
 import java.util.Calendar;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import javax.annotation.Nullable;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
 import org.apache.jackrabbit.oak.plugins.index.IndexingContext;
+import org.apache.jackrabbit.oak.plugins.index.lucene.binary.BinaryTextExtractor;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.FacetHelper;
 import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriter;
 import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriterFactory;
@@ -40,16 +36,10 @@ import org.apache.jackrabbit.oak.stats.C
 import org.apache.jackrabbit.oak.util.PerfLogger;
 import org.apache.jackrabbit.util.ISO8601;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.tika.config.TikaConfig;
-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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition.INDEX_DEFINITION_NODE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PROP_REFRESH_DEFN;
 
@@ -61,12 +51,8 @@ public class LuceneIndexEditorContext {
     private static final PerfLogger PERF_LOGGER =
             new PerfLogger(LoggerFactory.getLogger(LuceneIndexEditorContext.class.getName() + ".perf"));
 
-
-
     private FacetsConfig facetsConfig;
 
-    private static final Parser defaultParser = createDefaultParser();
-
     private IndexDefinition definition;
 
     private final NodeBuilder definitionBuilder;
@@ -81,10 +67,6 @@ public class LuceneIndexEditorContext {
 
     private boolean reindex;
 
-    private Parser parser;
-
-    private final TextExtractionStats textExtractionStats = new TextExtractionStats();
-
     private final ExtractedTextCache extractedTextCache;
 
     private final IndexAugmentorFactory augmentorFactory;
@@ -94,10 +76,6 @@ public class LuceneIndexEditorContext {
     private final IndexingContext indexingContext;
 
     private final boolean asyncIndexing;
-    /**
-     * The media types supported by the parser used.
-     */
-    private Set<MediaType> supportedMediaTypes;
 
     //Intentionally static, so that it can be set without passing around clock objects
     //Set for testing ONLY
@@ -105,6 +83,8 @@ public class LuceneIndexEditorContext {
 
     private final boolean indexDefnRewritten;
 
+    private BinaryTextExtractor textExtractor;
+
     LuceneIndexEditorContext(NodeState root, NodeBuilder definition,
                              @Nullable IndexDefinition indexDefinition,
                              IndexUpdateCallback updateCallback,
@@ -131,13 +111,6 @@ public class LuceneIndexEditorContext {
         }
     }
 
-    Parser getParser() {
-        if (parser == null){
-            parser = initializeTikaParser(definition);
-        }
-        return parser;
-    }
-
     LuceneIndexWriter getWriter() throws IOException {
         if (writer == null) {
             //Lazy initialization so as to ensure that definition is based
@@ -170,8 +143,9 @@ public class LuceneIndexEditorContext {
 
             PERF_LOGGER.end(start, -1, "Overall Closed IndexWriter for directory {}", definition);
 
-            textExtractionStats.log(reindex);
-            textExtractionStats.collectStats(extractedTextCache);
+            if (textExtractor != null){
+                textExtractor.done(reindex);
+            }
         }
     }
     /** Only set for testing */
@@ -222,13 +196,6 @@ public class LuceneIndexEditorContext {
         return indexedNodes;
     }
 
-    public boolean isSupportedMediaType(String type) {
-        if (supportedMediaTypes == null) {
-            supportedMediaTypes = getParser().getSupportedTypes(new ParseContext());
-        }
-        return supportedMediaTypes.contains(MediaType.parse(type));
-    }
-
     void indexUpdate() throws CommitFailedException {
         updateCallback.indexUpdate();
     }
@@ -244,18 +211,12 @@ public class LuceneIndexEditorContext {
         return facetsConfig;
     }
 
-    @Deprecated
-    public void recordTextExtractionStats(long timeInMillis, long bytesRead) {
-        //Keeping deprecated method to avoid major version change
-        recordTextExtractionStats(timeInMillis, bytesRead, 0);
-    }
-
-    public void recordTextExtractionStats(long timeInMillis, long bytesRead, int textLength) {
-        textExtractionStats.addStats(timeInMillis, bytesRead, textLength);
-    }
-
-    ExtractedTextCache getExtractedTextCache() {
-        return extractedTextCache;
+    BinaryTextExtractor getTextExtractor(){
+        if (textExtractor == null){
+            //Create lazily to ensure that if its reindex case then update definition is picked
+            textExtractor = new BinaryTextExtractor(extractedTextCache, definition, reindex);
+        }
+        return textExtractor;
     }
 
     IndexAugmentorFactory getAugmentorFactory() {
@@ -301,110 +262,4 @@ public class LuceneIndexEditorContext {
         }
         return new IndexDefinition(root, defnState,indexingContext.getIndexPath());
     }
-
-    private static Parser initializeTikaParser(IndexDefinition definition) {
-        ClassLoader current = Thread.currentThread().getContextClassLoader();
-        try {
-            if (definition.hasCustomTikaConfig()) {
-                log.debug("[{}] Using custom tika config", definition.getIndexName());
-                Thread.currentThread().setContextClassLoader(LuceneIndexEditorContext.class.getClassLoader());
-                InputStream is = definition.getTikaConfig();
-                try {
-                    return new AutoDetectParser(getTikaConfig(is, definition));
-                } finally {
-                    IOUtils.closeQuietly(is);
-                }
-            }
-        }finally {
-            Thread.currentThread().setContextClassLoader(current);
-        }
-        return defaultParser;
-    }
-
-    private static AutoDetectParser createDefaultParser() {
-        ClassLoader current = Thread.currentThread().getContextClassLoader();
-        URL configUrl = LuceneIndexEditorContext.class.getResource("tika-config.xml");
-        InputStream is = null;
-        if (configUrl != null) {
-            try {
-                Thread.currentThread().setContextClassLoader(LuceneIndexEditorContext.class.getClassLoader());
-                is = configUrl.openStream();
-                TikaConfig config = new TikaConfig(is);
-                log.info("Loaded default Tika Config from classpath {}", configUrl);
-                return new AutoDetectParser(config);
-            } catch (Exception e) {
-                log.warn("Tika configuration not available : " + configUrl, e);
-            } finally {
-                IOUtils.closeQuietly(is);
-                Thread.currentThread().setContextClassLoader(current);
-            }
-        } else {
-            log.warn("Default Tika configuration not found from {}", configUrl);
-        }
-        return new AutoDetectParser();
-    }
-
-    private static TikaConfig getTikaConfig(InputStream configStream, Object source){
-        try {
-            return new TikaConfig(configStream);
-        } catch (Exception e) {
-            log.warn("Tika configuration not available : "+source, e);
-        }
-        return TikaConfig.getDefaultConfig();
-    }
-
-    static class TextExtractionStats {
-        /**
-         * Log stats only if time spent is more than 2 min
-         */
-        private static final long LOGGING_THRESHOLD = TimeUnit.MINUTES.toMillis(1);
-        private int count;
-        private long totalBytesRead;
-        private long totalTime;
-        private long totalTextLength;
-
-        public void addStats(long timeInMillis, long bytesRead, int textLength) {
-            count++;
-            totalBytesRead += bytesRead;
-            totalTime += timeInMillis;
-            totalTextLength += textLength;
-        }
-
-        public void log(boolean reindex) {
-            if (log.isDebugEnabled()) {
-                log.debug("Text extraction stats {}", this);
-            } else if (anyParsingDone() && (reindex || isTakingLotsOfTime())) {
-                log.info("Text extraction stats {}", this);
-            }
-        }
-
-        public void collectStats(ExtractedTextCache cache){
-            cache.addStats(count, totalTime, totalBytesRead, totalTextLength);
-        }
-
-        private boolean isTakingLotsOfTime() {
-            return totalTime > LOGGING_THRESHOLD;
-        }
-
-        private boolean anyParsingDone() {
-            return count > 0;
-        }
-
-        @Override
-        public String toString() {
-            return String.format(" %d (Time Taken %s, Bytes Read %s, Extracted text size %s)",
-                    count,
-                    timeInWords(totalTime),
-                    humanReadableByteCount(totalBytesRead),
-                    humanReadableByteCount(totalTextLength));
-        }
-
-        private static String timeInWords(long millis) {
-            return String.format("%d min, %d sec",
-                    TimeUnit.MILLISECONDS.toMinutes(millis),
-                    TimeUnit.MILLISECONDS.toSeconds(millis) -
-                            TimeUnit.MINUTES.toSeconds(TimeUnit.MILLISECONDS.toMinutes(millis))
-            );
-        }
-    }
 }

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java?rev=1777676&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java Fri Jan  6 18:08:30 2017
@@ -0,0 +1,260 @@
+/*
+ * 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.jackrabbit.oak.plugins.index.lucene.binary;
+
+import java.io.InputStream;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.io.CountingInputStream;
+import org.apache.commons.io.IOUtils;
+import org.apache.jackrabbit.JcrConstants;
+import org.apache.jackrabbit.oak.api.Blob;
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.commons.io.LazyInputStream;
+import org.apache.jackrabbit.oak.plugins.index.fulltext.ExtractedText;
+import org.apache.jackrabbit.oak.plugins.index.lucene.ExtractedTextCache;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditorContext;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.util.BlobByteSource;
+import org.apache.lucene.document.Field;
+import org.apache.tika.config.TikaConfig;
+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.apache.tika.sax.WriteOutContentHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.jackrabbit.JcrConstants.JCR_DATA;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newFulltextField;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditor.TEXT_EXTRACTION_ERROR;
+
+public class BinaryTextExtractor {
+    private static final Logger log = LoggerFactory.getLogger(BinaryTextExtractor.class);
+    private static final Parser defaultParser = createDefaultParser();
+    private static final long SMALL_BINARY = Long.getLong("oak.lucene.smallBinary", 16 * 1024);
+    private final TextExtractionStats textExtractionStats = new TextExtractionStats();
+    private final ExtractedTextCache extractedTextCache;
+    private final IndexDefinition definition;
+    private final boolean reindex;
+    private Parser parser;
+    /**
+     * The media types supported by the parser used.
+     */
+    private Set<MediaType> supportedMediaTypes;
+
+    public BinaryTextExtractor(ExtractedTextCache extractedTextCache, IndexDefinition definition, boolean reindex) {
+        this.extractedTextCache = extractedTextCache;
+        this.definition = definition;
+        this.reindex = reindex;
+    }
+
+    public void done(boolean reindex){
+        textExtractionStats.log(reindex);
+        textExtractionStats.collectStats(extractedTextCache);
+    }
+
+    public List<Field> newBinary(
+            PropertyState property, NodeState state, String nodePath, String path) {
+        List<Field> fields = new ArrayList<Field>();
+        Metadata metadata = new Metadata();
+
+        //jcr:mimeType is mandatory for a binary to be indexed
+        String type = state.getString(JcrConstants.JCR_MIMETYPE);
+
+        if (type == null || !isSupportedMediaType(type)) {
+            log.trace(
+                    "[{}] Ignoring binary content for node {} due to unsupported (or null) jcr:mimeType [{}]",
+                    getIndexName(), path, type);
+            return fields;
+        }
+
+        metadata.set(Metadata.CONTENT_TYPE, type);
+        if (JCR_DATA.equals(property.getName())) {
+            String encoding = state.getString(JcrConstants.JCR_ENCODING);
+            if (encoding != null) { // not mandatory
+                metadata.set(Metadata.CONTENT_ENCODING, encoding);
+            }
+        }
+
+        for (Blob v : property.getValue(Type.BINARIES)) {
+            String value = parseStringValue(v, metadata, path, property.getName());
+            if (value == null){
+                continue;
+            }
+
+            if (nodePath != null){
+                fields.add(newFulltextField(nodePath, value, true));
+            } else {
+                fields.add(newFulltextField(value, true));
+            }
+        }
+        return fields;
+    }
+
+    private String parseStringValue(Blob v, Metadata metadata, String path, String propertyName) {
+        String text = extractedTextCache.get(path, propertyName, v, reindex);
+        if (text == null){
+            text = parseStringValue0(v, metadata, path);
+        }
+        return text;
+    }
+
+    private String parseStringValue0(Blob v, Metadata metadata, String path) {
+        WriteOutContentHandler handler = new WriteOutContentHandler(definition.getMaxExtractLength());
+        long start = System.currentTimeMillis();
+        long bytesRead = 0;
+        long length = v.length();
+        if (log.isDebugEnabled()) {
+            log.debug("Extracting {}, {} bytes, id {}", path, length, v.getContentIdentity());
+        }
+        String oldThreadName = null;
+        if (length > SMALL_BINARY) {
+            Thread t = Thread.currentThread();
+            oldThreadName = t.getName();
+            t.setName(oldThreadName + ": Extracting " + path + ", " + length + " bytes");
+        }
+        try {
+            CountingInputStream stream = new CountingInputStream(new LazyInputStream(new BlobByteSource(v)));
+            try {
+                getParser().parse(stream, handler, metadata, new ParseContext());
+            } finally {
+                bytesRead = stream.getCount();
+                stream.close();
+            }
+        } catch (LinkageError e) {
+            // Capture and ignore errors caused by extraction libraries
+            // not being present. This is equivalent to disabling
+            // selected media types in configuration, so we can simply
+            // ignore these errors.
+        } catch (Throwable t) {
+            // Capture and report any other full text extraction problems.
+            // The special STOP exception is used for normal termination.
+            if (!handler.isWriteLimitReached(t)) {
+                log.debug(
+                        "[{}] Failed to extract text from a binary property: {}."
+                                + " This is a fairly common case, and nothing to"
+                                + " worry about. The stack trace is included to"
+                                + " help improve the text extraction feature.",
+                        getIndexName(), path, t);
+                extractedTextCache.put(v, ExtractedText.ERROR);
+                return TEXT_EXTRACTION_ERROR;
+            }
+        } finally {
+            if (oldThreadName != null) {
+                Thread.currentThread().setName(oldThreadName);
+            }
+        }
+        String result = handler.toString();
+        if (bytesRead > 0) {
+            long time = System.currentTimeMillis() - start;
+            int len = result.length();
+            recordTextExtractionStats(time, bytesRead, len);
+            if (log.isDebugEnabled()) {
+                log.debug("Extracting {} took {} ms, {} bytes read, {} text size",
+                        path, time, bytesRead, len);
+            }
+        }
+        extractedTextCache.put(v,  new ExtractedText(ExtractedText.ExtractionResult.SUCCESS, result));
+        return result;
+    }
+
+    private void recordTextExtractionStats(long timeInMillis, long bytesRead, int textLength) {
+        textExtractionStats.addStats(timeInMillis, bytesRead, textLength);
+    }
+
+    private String getIndexName() {
+        return definition.getIndexName();
+    }
+
+    //~-------------------------------------------< Tika >
+
+    private Parser getParser() {
+        if (parser == null){
+            parser = initializeTikaParser(definition);
+        }
+        return parser;
+    }
+
+    private boolean isSupportedMediaType(String type) {
+        if (supportedMediaTypes == null) {
+            supportedMediaTypes = getParser().getSupportedTypes(new ParseContext());
+        }
+        return supportedMediaTypes.contains(MediaType.parse(type));
+    }
+
+    private static Parser initializeTikaParser(IndexDefinition definition) {
+        ClassLoader current = Thread.currentThread().getContextClassLoader();
+        try {
+            if (definition.hasCustomTikaConfig()) {
+                log.debug("[{}] Using custom tika config", definition.getIndexName());
+                Thread.currentThread().setContextClassLoader(LuceneIndexEditorContext.class.getClassLoader());
+                InputStream is = definition.getTikaConfig();
+                try {
+                    return new AutoDetectParser(getTikaConfig(is, definition));
+                } finally {
+                    IOUtils.closeQuietly(is);
+                }
+            }
+        }finally {
+            Thread.currentThread().setContextClassLoader(current);
+        }
+        return defaultParser;
+    }
+
+    private static AutoDetectParser createDefaultParser() {
+        ClassLoader current = Thread.currentThread().getContextClassLoader();
+        URL configUrl = LuceneIndexEditorContext.class.getResource("tika-config.xml");
+        InputStream is = null;
+        if (configUrl != null) {
+            try {
+                Thread.currentThread().setContextClassLoader(LuceneIndexEditorContext.class.getClassLoader());
+                is = configUrl.openStream();
+                TikaConfig config = new TikaConfig(is);
+                log.info("Loaded default Tika Config from classpath {}", configUrl);
+                return new AutoDetectParser(config);
+            } catch (Exception e) {
+                log.warn("Tika configuration not available : " + configUrl, e);
+            } finally {
+                IOUtils.closeQuietly(is);
+                Thread.currentThread().setContextClassLoader(current);
+            }
+        } else {
+            log.warn("Default Tika configuration not found");
+        }
+        return new AutoDetectParser();
+    }
+
+    private static TikaConfig getTikaConfig(InputStream configStream, Object source){
+        try {
+            return new TikaConfig(configStream);
+        } catch (Exception e) {
+            log.warn("Tika configuration not available : "+source, e);
+        }
+        return TikaConfig.getDefaultConfig();
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/BinaryTextExtractor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java?rev=1777676&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java Fri Jan  6 18:08:30 2017
@@ -0,0 +1,84 @@
+/*
+ * 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.jackrabbit.oak.plugins.index.lucene.binary;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.jackrabbit.oak.plugins.index.lucene.ExtractedTextCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount;
+
+class TextExtractionStats {
+    private static final Logger log = LoggerFactory.getLogger(TextExtractionStats.class);
+    /**
+     * Log stats only if time spent is more than 1 min
+     */
+    private static final long LOGGING_THRESHOLD = TimeUnit.MINUTES.toMillis(1);
+    private int count;
+    private long totalBytesRead;
+    private long totalTime;
+    private long totalTextLength;
+
+    public void addStats(long timeInMillis, long bytesRead, int textLength) {
+        count++;
+        totalBytesRead += bytesRead;
+        totalTime += timeInMillis;
+        totalTextLength += textLength;
+    }
+
+    public void log(boolean reindex) {
+        if (log.isDebugEnabled()) {
+            log.debug("Text extraction stats {}", this);
+        } else if (anyParsingDone() && (reindex || isTakingLotsOfTime())) {
+            log.info("Text extraction stats {}", this);
+        }
+    }
+
+    public void collectStats(ExtractedTextCache cache){
+        cache.addStats(count, totalTime, totalBytesRead, totalTextLength);
+    }
+
+    private boolean isTakingLotsOfTime() {
+        return totalTime > LOGGING_THRESHOLD;
+    }
+
+    private boolean anyParsingDone() {
+        return count > 0;
+    }
+
+    @Override
+    public String toString() {
+        return String.format(" %d (Time Taken %s, Bytes Read %s, Extracted text size %s)",
+                count,
+                timeInWords(totalTime),
+                humanReadableByteCount(totalBytesRead),
+                humanReadableByteCount(totalTextLength));
+    }
+
+    private static String timeInWords(long millis) {
+        return String.format("%d min, %d sec",
+                TimeUnit.MILLISECONDS.toMinutes(millis),
+                TimeUnit.MILLISECONDS.toSeconds(millis) -
+                        TimeUnit.MINUTES.toSeconds(TimeUnit.MILLISECONDS.toMinutes(millis))
+        );
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/binary/TextExtractionStats.java
------------------------------------------------------------------------------
    svn:eol-style = native