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 2016/08/04 09:46:54 UTC

svn commit: r1755161 - in /jackrabbit/oak/trunk/oak-lucene/src: main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/ test/java/org/apache/jackrabbit/oak/plugins/index/lucene/ test/ja...

Author: chetanm
Date: Thu Aug  4 09:46:54 2016
New Revision: 1755161

URL: http://svn.apache.org/viewvc?rev=1755161&view=rev
Log:
OAK-4566 - Multiplexing store support in Lucene Indexes

Refactor the logic in LuceneIndexEditorContext to move IndexWriter related code to DefaultIndexEditor. This is more or less a exact move i.e. code flow is not changed much.

Lazy initialization behaviour of the writer is still retained

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriter.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterFactory.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/IndexWriterUtils.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriter.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriterFactory.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java   (with props)
Modified:
    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
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProvider.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java

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=1755161&r1=1755160&r2=1755161&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 Thu Aug  4 09:46:54 2016
@@ -42,6 +42,8 @@ import org.apache.jackrabbit.oak.plugins
 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.writer.LuceneIndexWriter;
+import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriterFactory;
 import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
 import org.apache.jackrabbit.oak.plugins.memory.StringPropertyState;
 import org.apache.jackrabbit.oak.plugins.tree.TreeFactory;
@@ -59,8 +61,6 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.util.BytesRef;
 import org.apache.tika.metadata.Metadata;
 import org.apache.tika.parser.ParseContext;
@@ -72,7 +72,6 @@ import static org.apache.jackrabbit.JcrC
 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.*;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newPathTerm;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.util.ConfigUtil.getPrimaryTypeName;
 
 /**
@@ -126,14 +125,14 @@ public class LuceneIndexEditor implement
 
     LuceneIndexEditor(NodeState root, NodeBuilder definition,
                         IndexUpdateCallback updateCallback,
-                        @Nullable IndexCopier indexCopier,
+                        LuceneIndexWriterFactory writerFactory,
                         ExtractedTextCache extractedTextCache,
                       IndexAugmentorFactory augmentorFactory) throws CommitFailedException {
         this.parent = null;
         this.name = null;
         this.path = "/";
         this.context = new LuceneIndexEditorContext(root, definition,
-                updateCallback, indexCopier, extractedTextCache, augmentorFactory);
+                updateCallback, writerFactory, extractedTextCache, augmentorFactory);
         this.root = root;
         this.isDeleted = false;
         this.matcherState = MatcherState.NONE;
@@ -272,10 +271,9 @@ public class LuceneIndexEditor implement
             // tree deletion is handled on the parent node
             String path = concat(getPath(), name);
             try {
-                IndexWriter writer = context.getWriter();
+                LuceneIndexWriter writer = context.getWriter();
                 // Remove all index entries in the removed subtree
-                writer.deleteDocuments(newPathTerm(path));
-                writer.deleteDocuments(new PrefixQuery(newPathTerm(path + "/")));
+                writer.deleteDocuments(path);
                 this.context.indexUpdate();
             } catch (IOException e) {
                 throw new CommitFailedException("Lucene", 5,
@@ -300,7 +298,7 @@ public class LuceneIndexEditor implement
                     log.trace("[{}] Indexed document for {} is {}", getIndexName(), path, d);
                 }
                 context.indexUpdate();
-                context.getWriter().updateDocument(newPathTerm(path), d);
+                context.getWriter().updateDocument(path, d);
                 return true;
             }
         } catch (IOException e) {

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=1755161&r1=1755160&r2=1755161&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 Thu Aug  4 09:46:54 2016
@@ -16,40 +16,26 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
-import javax.annotation.Nonnull;
-import javax.annotation.Nullable;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
 import java.util.Calendar;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.IOUtils;
 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.plugins.index.IndexUpdateCallback;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.FacetHelper;
-import org.apache.jackrabbit.oak.plugins.index.lucene.util.SuggestHelper;
+import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriter;
+import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriterFactory;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.apache.jackrabbit.oak.stats.Clock;
 import org.apache.jackrabbit.oak.util.PerfLogger;
 import org.apache.jackrabbit.util.ISO8601;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
-import org.apache.lucene.analysis.shingle.ShingleAnalyzerWrapper;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
 import org.apache.tika.config.TikaConfig;
 import org.apache.tika.mime.MediaType;
 import org.apache.tika.parser.AutoDetectParser;
@@ -60,10 +46,6 @@ 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.LuceneIndexConstants.PERSISTENCE_PATH;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.SUGGEST_DATA_CHILD_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.VERSION;
-import static org.apache.lucene.store.NoLockFactory.getNoLockFactory;
 
 public class LuceneIndexEditorContext {
 
@@ -75,63 +57,15 @@ public class LuceneIndexEditorContext {
 
     private final FacetsConfig facetsConfig;
 
-    static IndexWriterConfig getIndexWriterConfig(IndexDefinition definition, boolean remoteDir) {
-        // FIXME: Hack needed to make Lucene work in an OSGi environment
-        Thread thread = Thread.currentThread();
-        ClassLoader loader = thread.getContextClassLoader();
-        thread.setContextClassLoader(IndexWriterConfig.class.getClassLoader());
-        try {
-            Analyzer definitionAnalyzer = definition.getAnalyzer();
-            Map<String, Analyzer> analyzers = new HashMap<String, Analyzer>();
-            analyzers.put(FieldNames.SPELLCHECK, new ShingleAnalyzerWrapper(LuceneIndexConstants.ANALYZER, 3));
-            if (!definition.isSuggestAnalyzed()) {
-                analyzers.put(FieldNames.SUGGEST, SuggestHelper.getAnalyzer());
-            }
-            Analyzer analyzer = new PerFieldAnalyzerWrapper(definitionAnalyzer, analyzers);
-            IndexWriterConfig config = new IndexWriterConfig(VERSION, analyzer);
-            if (remoteDir) {
-                config.setMergeScheduler(new SerialMergeScheduler());
-            }
-            if (definition.getCodec() != null) {
-                config.setCodec(definition.getCodec());
-            }
-            return config;
-        } finally {
-            thread.setContextClassLoader(loader);
-        }
-    }
-
-    static Directory newIndexDirectory(IndexDefinition indexDefinition, NodeBuilder definition)
-            throws IOException {
-        String path = definition.getString(PERSISTENCE_PATH);
-        if (path == null) {
-            return new OakDirectory(definition, indexDefinition, false);
-        } else {
-            // try {
-            File file = new File(path);
-            file.mkdirs();
-            // TODO: close() is never called
-            // TODO: no locking used
-            // --> using the FS backend for the index is in any case
-            // troublesome in clustering scenarios and for backup
-            // etc. so instead of fixing these issues we'd better
-            // work on making the in-content index work without
-            // problems (or look at the Solr indexer as alternative)
-            return FSDirectory.open(file, getNoLockFactory());
-            // } catch (IOException e) {
-            // throw new CommitFailedException("Lucene", 1,
-            // "Failed to open the index in " + path, e);
-            // }
-        }
-    }
-
     private static final Parser defaultParser = createDefaultParser();
 
     private IndexDefinition definition;
 
     private final NodeBuilder definitionBuilder;
 
-    private IndexWriter writer = null;
+    private final LuceneIndexWriterFactory indexWriterFactory;
+
+    private LuceneIndexWriter writer = null;
 
     private long indexedNodes;
 
@@ -141,11 +75,6 @@ public class LuceneIndexEditorContext {
 
     private Parser parser;
 
-    @Nullable
-    private final IndexCopier indexCopier;
-
-    private Directory directory;
-
     private final TextExtractionStats textExtractionStats = new TextExtractionStats();
 
     private final ExtractedTextCache extractedTextCache;
@@ -163,12 +92,13 @@ public class LuceneIndexEditorContext {
     private static Clock clock = Clock.SIMPLE;
 
     LuceneIndexEditorContext(NodeState root, NodeBuilder definition, IndexUpdateCallback updateCallback,
-                             @Nullable IndexCopier indexCopier, ExtractedTextCache extractedTextCache,
+                             LuceneIndexWriterFactory indexWriterFactory,
+                             ExtractedTextCache extractedTextCache,
                              IndexAugmentorFactory augmentorFactory) {
         configureUniqueId(definition);
         this.root = root;
         this.definitionBuilder = definition;
-        this.indexCopier = indexCopier;
+        this.indexWriterFactory = indexWriterFactory;
         this.definition = new IndexDefinition(root, definition);
         this.indexedNodes = 0;
         this.updateCallback = updateCallback;
@@ -187,99 +117,31 @@ public class LuceneIndexEditorContext {
         return parser;
     }
 
-    IndexWriter getWriter() throws IOException {
+    LuceneIndexWriter getWriter() throws IOException {
         if (writer == null) {
-            final long start = PERF_LOGGER.start();
-            directory = newIndexDirectory(definition, definitionBuilder);
-            IndexWriterConfig config;
-            if (indexCopier != null){
-                directory = indexCopier.wrapForWrite(definition, directory, reindex, LuceneIndexConstants.INDEX_DATA_CHILD_NAME);
-                config = getIndexWriterConfig(definition, false);
-            } else {
-                config = getIndexWriterConfig(definition, true);
-            }
-            writer = new IndexWriter(directory, config);
-            PERF_LOGGER.end(start, -1, "Created IndexWriter for directory {}", definition);
+            //Lazy initialization so as to ensure that definition is based
+            //on latest NodeBuilder state specially in case of reindexing
+            writer = indexWriterFactory.newInstance(definition, definitionBuilder, reindex);
         }
         return writer;
     }
 
-    private static void trackIndexSizeInfo(@Nonnull IndexWriter writer,
-                                           @Nonnull IndexDefinition definition,
-                                           @Nonnull Directory directory) throws IOException {
-        checkNotNull(writer);
-        checkNotNull(definition);
-        checkNotNull(directory);
-
-        int docs = writer.numDocs();
-        int ram = writer.numRamDocs();
-
-        log.trace("Writer for direcory {} - docs: {}, ramDocs: {}", definition, docs, ram);
-
-        String[] files = directory.listAll();
-        long overallSize = 0;
-        StringBuilder sb = new StringBuilder();
-        for (String f : files) {
-            sb.append(f).append(":");
-            if (directory.fileExists(f)) {
-                long size = directory.fileLength(f);
-                overallSize += size;
-                sb.append(size);
-            } else {
-                sb.append("--");
-            }
-            sb.append(", ");
-        }
-        log.trace("Directory overall size: {}, files: {}",
-            org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount(overallSize),
-            sb.toString());
-    }
-
     /**
      * close writer if it's not null
      */
     void closeWriter() throws IOException {
-        //If reindex or fresh index and write is null on close
-        //it indicates that the index is empty. In such a case trigger
-        //creation of write such that an empty Lucene index state is persisted
-        //in directory
-        if (reindex && writer == null){
-            getWriter();
-        }
-
-        boolean updateSuggestions = shouldUpdateSuggestions();
-        if (writer == null && updateSuggestions) {
-            log.debug("Would update suggester dictionary although no index changes were detected in current cycle");
-            getWriter();
-        }
+        Calendar currentTime = getCalendar();
+        final long start = PERF_LOGGER.start();
+        boolean indexUpdated = getWriter().close(currentTime.getTimeInMillis());
+        PERF_LOGGER.end(start, -1, "Closed writer for directory {}", definition);
 
-        if (writer != null) {
-            if (log.isTraceEnabled()) {
-                trackIndexSizeInfo(writer, definition, directory);
-            }
-
-            final long start = PERF_LOGGER.start();
-
-            Calendar lastUpdated = null;
-            if (updateSuggestions) {
-                lastUpdated = updateSuggester(writer.getAnalyzer());
-                PERF_LOGGER.end(start, -1, "Completed suggester for directory {}", definition);
-            }
-            if (lastUpdated == null) {
-                lastUpdated = getCalendar();
-            }
-
-            writer.close();
-            PERF_LOGGER.end(start, -1, "Closed writer for directory {}", definition);
-
-            directory.close();
-            PERF_LOGGER.end(start, -1, "Closed directory for directory {}", definition);
 
+        if (indexUpdated) {
             //OAK-2029 Record the last updated status so
             //as to make IndexTracker detect changes when index
             //is stored in file system
             NodeBuilder status = definitionBuilder.child(":status");
-            status.setProperty("lastUpdated", ISO8601.format(lastUpdated), Type.DATE);
+            status.setProperty("lastUpdated", ISO8601.format(currentTime), Type.DATE);
             status.setProperty("indexedNodes", indexedNodes);
 
             PERF_LOGGER.end(start, -1, "Overall Closed IndexWriter for directory {}", definition);
@@ -288,77 +150,6 @@ public class LuceneIndexEditorContext {
             textExtractionStats.collectStats(extractedTextCache);
         }
     }
-
-    /**
-     * eventually update suggest dictionary
-     * @throws IOException if suggest dictionary update fails
-     * @param analyzer the analyzer used to update the suggester
-     * @return {@link Calendar} object representing the lastUpdated value written by suggestions
-     */
-    private Calendar updateSuggester(Analyzer analyzer) throws IOException {
-        Calendar ret = null;
-        NodeBuilder suggesterStatus = definitionBuilder.child(SUGGEST_DATA_CHILD_NAME);
-        DirectoryReader reader = DirectoryReader.open(writer, false);
-        final OakDirectory suggestDirectory = new OakDirectory(definitionBuilder, SUGGEST_DATA_CHILD_NAME, definition, false);
-        try {
-            SuggestHelper.updateSuggester(suggestDirectory, analyzer, reader);
-            ret = getCalendar();
-            suggesterStatus.setProperty("lastUpdated", ISO8601.format(ret), Type.DATE);
-        } catch (Throwable e) {
-            log.warn("could not update suggester", e);
-        } finally {
-            suggestDirectory.close();
-            reader.close();
-        }
-
-        return ret;
-    }
-
-    /**
-     * Checks if last suggestion build time was done sufficiently in the past AND that there were non-zero indexedNodes
-     * stored in the last run. Note, if index is updated only to rebuild suggestions, even then we update indexedNodes,
-     * which would be zero in case it was a forced update of suggestions.
-     * @return is suggest dict should be updated
-     */
-    private boolean shouldUpdateSuggestions() {
-        boolean updateSuggestions = false;
-
-        if (definition.isSuggestEnabled()) {
-            NodeBuilder suggesterStatus = definitionBuilder.child(SUGGEST_DATA_CHILD_NAME);
-
-            PropertyState suggesterLastUpdatedValue = suggesterStatus.getProperty("lastUpdated");
-
-            if (suggesterLastUpdatedValue != null) {
-                Calendar suggesterLastUpdatedTime = ISO8601.parse(suggesterLastUpdatedValue.getValue(Type.DATE));
-
-                int updateFrequency = definition.getSuggesterUpdateFrequencyMinutes();
-                Calendar nextSuggestUpdateTime = (Calendar)suggesterLastUpdatedTime.clone();
-                nextSuggestUpdateTime.add(Calendar.MINUTE, updateFrequency);
-                if (getCalendar().after(nextSuggestUpdateTime)) {
-                    updateSuggestions = (writer != null || isIndexUpdatedAfter(suggesterLastUpdatedTime));
-                }
-            } else {
-                updateSuggestions = true;
-            }
-        }
-
-        return updateSuggestions;
-    }
-
-    /**
-     * @return {@code false} if persisted lastUpdated time for index is after {@code calendar}. {@code true} otherwise
-     */
-    private boolean isIndexUpdatedAfter(Calendar calendar) {
-        NodeBuilder indexStats = definitionBuilder.child(":status");
-        PropertyState indexLastUpdatedValue = indexStats.getProperty("lastUpdated");
-        if (indexLastUpdatedValue != null) {
-            Calendar indexLastUpdatedTime = ISO8601.parse(indexLastUpdatedValue.getValue(Type.DATE));
-            return indexLastUpdatedTime.after(calendar);
-        } else {
-            return true;
-        }
-    }
-
     /** Only set for testing */
     static void setClock(Clock c) {
         checkNotNull(c);

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProvider.java?rev=1755161&r1=1755160&r2=1755161&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProvider.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorProvider.java Thu Aug  4 09:46:54 2016
@@ -23,6 +23,7 @@ import org.apache.jackrabbit.oak.api.Com
 import org.apache.jackrabbit.oak.plugins.index.IndexEditor;
 import org.apache.jackrabbit.oak.plugins.index.IndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
+import org.apache.jackrabbit.oak.plugins.index.lucene.writer.DefaultIndexWriterFactory;
 import org.apache.jackrabbit.oak.spi.commit.Editor;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
@@ -69,7 +70,8 @@ public class LuceneIndexEditorProvider i
             @Nonnull IndexUpdateCallback callback)
             throws CommitFailedException {
         if (TYPE_LUCENE.equals(type)) {
-            return new LuceneIndexEditor(root, definition, callback, indexCopier, extractedTextCache, augmentorFactory);
+            return new LuceneIndexEditor(root, definition, callback, new DefaultIndexWriterFactory(indexCopier),
+                    extractedTextCache, augmentorFactory);
         }
         return null;
     }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java?rev=1755161&r1=1755160&r2=1755161&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/OakDirectory.java Thu Aug  4 09:46:54 2016
@@ -66,7 +66,7 @@ import static org.apache.jackrabbit.oak.
  * Implementation of the Lucene {@link Directory} (a flat list of files)
  * based on an Oak {@link NodeBuilder}.
  */
-class OakDirectory extends Directory {
+public class OakDirectory extends Directory {
     static final PerfLogger PERF_LOGGER = new PerfLogger(LoggerFactory.getLogger(OakDirectory.class.getName() + ".perf"));
     static final String PROP_DIR_LISTING = "dirListing";
     static final String PROP_BLOB_SIZE = "blobSize";

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriter.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriter.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriter.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriter.java Thu Aug  4 09:46:54 2016
@@ -0,0 +1,240 @@
+/*
+ * 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.writer;
+
+import java.io.IOException;
+import java.util.Calendar;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexCopier;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants;
+import org.apache.jackrabbit.oak.plugins.index.lucene.OakDirectory;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.SuggestHelper;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.util.PerfLogger;
+import org.apache.jackrabbit.util.ISO8601;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.store.Directory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.SUGGEST_DATA_CHILD_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newPathTerm;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.writer.IndexWriterUtils.getIndexWriterConfig;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.writer.IndexWriterUtils.newIndexDirectory;
+
+class DefaultIndexWriter implements LuceneIndexWriter {
+    private static final Logger log = LoggerFactory.getLogger(DefaultIndexWriter.class);
+    private static final PerfLogger PERF_LOGGER =
+            new PerfLogger(LoggerFactory.getLogger(LuceneIndexWriter.class.getName() + ".perf"));
+
+    private final IndexDefinition definition;
+    private final NodeBuilder definitionBuilder;
+    private final IndexCopier indexCopier;
+    private final boolean reindex;
+    private IndexWriter writer;
+    private Directory directory;
+
+    public DefaultIndexWriter(IndexDefinition definition, NodeBuilder definitionBuilder,
+                              @Nullable IndexCopier indexCopier, boolean reindex){
+        this.definition = definition;
+        this.definitionBuilder = definitionBuilder;
+        this.indexCopier = indexCopier;
+        this.reindex = reindex;
+    }
+
+    @Override
+    public void updateDocument(String path, Iterable<? extends IndexableField> doc) throws IOException {
+        getWriter().updateDocument(newPathTerm(path), doc);
+    }
+
+    @Override
+    public void deleteDocuments(String path) throws IOException {
+        getWriter().deleteDocuments(newPathTerm(path));
+        getWriter().deleteDocuments(new PrefixQuery(newPathTerm(path + "/")));
+    }
+
+    @Override
+    public boolean close(long timestamp) throws IOException {
+        //If reindex or fresh index and write is null on close
+        //it indicates that the index is empty. In such a case trigger
+        //creation of write such that an empty Lucene index state is persisted
+        //in directory
+        boolean indexUpdated = false;
+        if (reindex && writer == null){
+            getWriter();
+        }
+
+        Calendar currentTime = Calendar.getInstance();
+        currentTime.setTimeInMillis(timestamp);
+        boolean updateSuggestions = shouldUpdateSuggestions(currentTime);
+        if (writer == null && updateSuggestions) {
+            log.debug("Would update suggester dictionary although no index changes were detected in current cycle");
+            getWriter();
+        }
+
+        if (writer != null) {
+            indexUpdated = true;
+            if (log.isTraceEnabled()) {
+                trackIndexSizeInfo(writer, definition, directory);
+            }
+
+            final long start = PERF_LOGGER.start();
+
+            if (updateSuggestions) {
+                updateSuggester(writer.getAnalyzer(), currentTime);
+                PERF_LOGGER.end(start, -1, "Completed suggester for directory {}", definition);
+            }
+
+            writer.close();
+            PERF_LOGGER.end(start, -1, "Closed writer for directory {}", definition);
+
+            directory.close();
+            PERF_LOGGER.end(start, -1, "Closed directory for directory {}", definition);
+        }
+        return indexUpdated;
+    }
+
+    //~----------------------------------------< internal >
+
+    private IndexWriter getWriter() throws IOException {
+        if (writer == null) {
+            final long start = PERF_LOGGER.start();
+            directory = newIndexDirectory(definition, definitionBuilder);
+            IndexWriterConfig config;
+            if (indexCopier != null){
+                directory = indexCopier.wrapForWrite(definition, directory, reindex, LuceneIndexConstants.INDEX_DATA_CHILD_NAME);
+                config = getIndexWriterConfig(definition, false);
+            } else {
+                config = getIndexWriterConfig(definition, true);
+            }
+            writer = new IndexWriter(directory, config);
+            PERF_LOGGER.end(start, -1, "Created IndexWriter for directory {}", definition);
+        }
+        return writer;
+    }
+
+    /**
+     * eventually update suggest dictionary
+     * @throws IOException if suggest dictionary update fails
+     * @param analyzer the analyzer used to update the suggester
+     */
+    private void updateSuggester(Analyzer analyzer, Calendar currentTime) throws IOException {
+        NodeBuilder suggesterStatus = definitionBuilder.child(SUGGEST_DATA_CHILD_NAME);
+        DirectoryReader reader = DirectoryReader.open(writer, false);
+        final OakDirectory suggestDirectory = new OakDirectory(definitionBuilder, SUGGEST_DATA_CHILD_NAME, definition, false);
+        try {
+            SuggestHelper.updateSuggester(suggestDirectory, analyzer, reader);
+            suggesterStatus.setProperty("lastUpdated", ISO8601.format(currentTime), Type.DATE);
+        } catch (Throwable e) {
+            log.warn("could not update suggester", e);
+        } finally {
+            suggestDirectory.close();
+            reader.close();
+        }
+    }
+
+    /**
+     * Checks if last suggestion build time was done sufficiently in the past AND that there were non-zero indexedNodes
+     * stored in the last run. Note, if index is updated only to rebuild suggestions, even then we update indexedNodes,
+     * which would be zero in case it was a forced update of suggestions.
+     * @return is suggest dict should be updated
+     */
+    private boolean shouldUpdateSuggestions(Calendar currentTime) {
+        boolean updateSuggestions = false;
+
+        if (definition.isSuggestEnabled()) {
+            NodeBuilder suggesterStatus = definitionBuilder.child(SUGGEST_DATA_CHILD_NAME);
+
+            PropertyState suggesterLastUpdatedValue = suggesterStatus.getProperty("lastUpdated");
+
+            if (suggesterLastUpdatedValue != null) {
+                Calendar suggesterLastUpdatedTime = ISO8601.parse(suggesterLastUpdatedValue.getValue(Type.DATE));
+
+                int updateFrequency = definition.getSuggesterUpdateFrequencyMinutes();
+                Calendar nextSuggestUpdateTime = (Calendar)suggesterLastUpdatedTime.clone();
+                nextSuggestUpdateTime.add(Calendar.MINUTE, updateFrequency);
+                if (currentTime.after(nextSuggestUpdateTime)) {
+                    updateSuggestions = (writer != null || isIndexUpdatedAfter(suggesterLastUpdatedTime));
+                }
+            } else {
+                updateSuggestions = true;
+            }
+        }
+
+        return updateSuggestions;
+    }
+
+    /**
+     * @return {@code false} if persisted lastUpdated time for index is after {@code calendar}. {@code true} otherwise
+     */
+    private boolean isIndexUpdatedAfter(Calendar calendar) {
+        NodeBuilder indexStats = definitionBuilder.child(":status");
+        PropertyState indexLastUpdatedValue = indexStats.getProperty("lastUpdated");
+        if (indexLastUpdatedValue != null) {
+            Calendar indexLastUpdatedTime = ISO8601.parse(indexLastUpdatedValue.getValue(Type.DATE));
+            return indexLastUpdatedTime.after(calendar);
+        } else {
+            return true;
+        }
+    }
+
+    private static void trackIndexSizeInfo(@Nonnull IndexWriter writer,
+                                           @Nonnull IndexDefinition definition,
+                                           @Nonnull Directory directory) throws IOException {
+        checkNotNull(writer);
+        checkNotNull(definition);
+        checkNotNull(directory);
+
+        int docs = writer.numDocs();
+        int ram = writer.numRamDocs();
+
+        log.trace("Writer for directory {} - docs: {}, ramDocs: {}", definition, docs, ram);
+
+        String[] files = directory.listAll();
+        long overallSize = 0;
+        StringBuilder sb = new StringBuilder();
+        for (String f : files) {
+            sb.append(f).append(":");
+            if (directory.fileExists(f)) {
+                long size = directory.fileLength(f);
+                overallSize += size;
+                sb.append(size);
+            } else {
+                sb.append("--");
+            }
+            sb.append(", ");
+        }
+        log.trace("Directory overall size: {}, files: {}",
+                org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount(overallSize),
+                sb.toString());
+    }
+}

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

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterFactory.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterFactory.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterFactory.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterFactory.java Thu Aug  4 09:46:54 2016
@@ -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.jackrabbit.oak.plugins.index.lucene.writer;
+
+import javax.annotation.Nullable;
+
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexCopier;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+public class DefaultIndexWriterFactory implements LuceneIndexWriterFactory {
+    private final IndexCopier indexCopier;
+
+    public DefaultIndexWriterFactory(@Nullable IndexCopier indexCopier) {
+        this.indexCopier = indexCopier;
+    }
+
+    @Override
+    public LuceneIndexWriter newInstance(IndexDefinition definition,
+                                         NodeBuilder definitionBuilder, boolean reindex) {
+        return new DefaultIndexWriter(definition, definitionBuilder, indexCopier, reindex);
+    }
+}

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

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/IndexWriterUtils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/IndexWriterUtils.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/IndexWriterUtils.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/IndexWriterUtils.java Thu Aug  4 09:46:54 2016
@@ -0,0 +1,97 @@
+/*
+ * 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.writer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants;
+import org.apache.jackrabbit.oak.plugins.index.lucene.OakDirectory;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.SuggestHelper;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
+import org.apache.lucene.analysis.shingle.ShingleAnalyzerWrapper;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_PATH;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.VERSION;
+import static org.apache.lucene.store.NoLockFactory.getNoLockFactory;
+
+public class IndexWriterUtils {
+
+    public static IndexWriterConfig getIndexWriterConfig(IndexDefinition definition, boolean remoteDir) {
+        // FIXME: Hack needed to make Lucene work in an OSGi environment
+        Thread thread = Thread.currentThread();
+        ClassLoader loader = thread.getContextClassLoader();
+        thread.setContextClassLoader(IndexWriterConfig.class.getClassLoader());
+        try {
+            Analyzer definitionAnalyzer = definition.getAnalyzer();
+            Map<String, Analyzer> analyzers = new HashMap<String, Analyzer>();
+            analyzers.put(FieldNames.SPELLCHECK, new ShingleAnalyzerWrapper(LuceneIndexConstants.ANALYZER, 3));
+            if (!definition.isSuggestAnalyzed()) {
+                analyzers.put(FieldNames.SUGGEST, SuggestHelper.getAnalyzer());
+            }
+            Analyzer analyzer = new PerFieldAnalyzerWrapper(definitionAnalyzer, analyzers);
+            IndexWriterConfig config = new IndexWriterConfig(VERSION, analyzer);
+            if (remoteDir) {
+                config.setMergeScheduler(new SerialMergeScheduler());
+            }
+            if (definition.getCodec() != null) {
+                config.setCodec(definition.getCodec());
+            }
+            return config;
+        } finally {
+            thread.setContextClassLoader(loader);
+        }
+    }
+
+    public static Directory newIndexDirectory(IndexDefinition indexDefinition, NodeBuilder definition)
+            throws IOException {
+        String path = definition.getString(PERSISTENCE_PATH);
+        if (path == null) {
+            return new OakDirectory(definition, indexDefinition, false);
+        } else {
+            // try {
+            File file = new File(path);
+            file.mkdirs();
+            // TODO: close() is never called
+            // TODO: no locking used
+            // --> using the FS backend for the index is in any case
+            // troublesome in clustering scenarios and for backup
+            // etc. so instead of fixing these issues we'd better
+            // work on making the in-content index work without
+            // problems (or look at the Solr indexer as alternative)
+            return FSDirectory.open(file, getNoLockFactory());
+            // } catch (IOException e) {
+            // throw new CommitFailedException("Lucene", 1,
+            // "Failed to open the index in " + path, e);
+            // }
+        }
+    }
+
+}

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

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriter.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriter.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriter.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriter.java Thu Aug  4 09:46:54 2016
@@ -0,0 +1,50 @@
+/*
+ * 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.writer;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexableField;
+
+public interface LuceneIndexWriter {
+
+    /**
+     * Updates the Lucene document having given path
+     *
+     * @param path path of the NodeState which the Document represents
+     * @param doc updated document
+     */
+    void updateDocument(String path, Iterable<? extends IndexableField> doc) throws IOException;
+
+    /**
+     * Deletes Lucene Documents which are same or child of given path
+     *
+     * @param path path whose children need to be deleted
+     */
+    void deleteDocuments(String path) throws IOException;
+
+    /**
+     * Closes the underlying writer.
+     *
+     * @param timestamp timestamp to be used for recording at status in NodeBuilder
+     * @return true if index was updated or any write happened.
+     */
+    boolean close(long timestamp) throws IOException;
+}

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

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriterFactory.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriterFactory.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriterFactory.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/LuceneIndexWriterFactory.java Thu Aug  4 09:46:54 2016
@@ -0,0 +1,29 @@
+/*
+ * 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.writer;
+
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+public interface LuceneIndexWriterFactory {
+
+    LuceneIndexWriter newInstance(IndexDefinition definition, NodeBuilder definitionBuilder, boolean reindex);
+
+}

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

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java?rev=1755161&r1=1755160&r2=1755161&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java Thu Aug  4 09:46:54 2016
@@ -46,13 +46,13 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_FILE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_PATH;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditorContext.getIndexWriterConfig;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditorContext.newIndexDirectory;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.TestUtil.NT_TEST;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.TestUtil.createNodeWithType;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.TestUtil.newLuceneIndexDefinitionV2;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper.newLuceneIndexDefinition;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper.newLucenePropertyIndexDefinition;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.writer.IndexWriterUtils.getIndexWriterConfig;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.writer.IndexWriterUtils.newIndexDirectory;
 import static org.apache.jackrabbit.oak.plugins.memory.PropertyStates.createProperty;
 import static org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent.INITIAL_CONTENT;
 

Added: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java?rev=1755161&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java Thu Aug  4 09:46:54 2016
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.jackrabbit.oak.plugins.index.lucene.writer;
+
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.lucene.document.Document;
+import org.junit.Test;
+
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newPathField;
+import static org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent.INITIAL_CONTENT;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class DefaultIndexWriterTest {
+    private NodeState root = INITIAL_CONTENT;
+
+    private NodeBuilder builder = root.builder();
+
+    @Test
+    public void lazyInit() throws Exception {
+        IndexDefinition defn = new IndexDefinition(root, builder.getNodeState());
+        DefaultIndexWriter writer = new DefaultIndexWriter(defn, builder, null, false);
+        assertFalse(writer.close(0));
+    }
+
+    @Test
+    public void writeInitializedUponReindex() throws Exception {
+        IndexDefinition defn = new IndexDefinition(root, builder.getNodeState());
+        DefaultIndexWriter writer = new DefaultIndexWriter(defn, builder, null, true);
+        assertTrue(writer.close(0));
+    }
+
+    @Test
+    public void indexUpdated() throws Exception {
+        IndexDefinition defn = new IndexDefinition(root, builder.getNodeState());
+        DefaultIndexWriter writer = new DefaultIndexWriter(defn, builder, null, false);
+
+        Document document = new Document();
+        document.add(newPathField("/a/b"));
+
+        writer.updateDocument("/a/b", document);
+
+        assertTrue(writer.close(0));
+    }
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/writer/DefaultIndexWriterTest.java
------------------------------------------------------------------------------
    svn:eol-style = native