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/07 10:10:07 UTC

svn commit: r1777758 - in /jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene: LuceneDocumentMaker.java LuceneIndexEditor.java LuceneIndexEditorContext.java util/FacetsConfigProvider.java

Author: chetanm
Date: Sat Jan  7 10:10:07 2017
New Revision: 1777758

URL: http://svn.apache.org/viewvc?rev=1777758&view=rev
Log:
OAK-5414 - Refactor LuceneIndexEditor to move out logic related to making Lucene Document to separate class

-- Moved out the makeDocument logic to LuceneDocumentMaker
-- For FacetsConfig introduced a FacetsConfigProvider which would lazily created config node in index definition builder.
-- For non async case it would be null and due to that faceting logic would be disabled
-- For non async case BinaryTextExtractor would be null and due to that faceting logic would be disabled

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneDocumentMaker.java
      - copied, changed from r1777757, 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/util/FacetsConfigProvider.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

Copied: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneDocumentMaker.java (from r1777757, 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/LuceneDocumentMaker.java?p2=jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneDocumentMaker.java&p1=jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java&r1=1777757&r2=1777758&rev=1777758&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/LuceneDocumentMaker.java Sat Jan  7 10:10:07 2017
@@ -1,19 +1,22 @@
 /*
- * 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
+ * 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
+ *   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.
+ * 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;
 
 import java.io.IOException;
@@ -21,24 +24,19 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import javax.annotation.Nullable;
+
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-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.plugins.index.IndexEditor;
-import org.apache.jackrabbit.oak.plugins.index.PathFilter;
-import org.apache.jackrabbit.oak.plugins.index.lucene.Aggregate.Matcher;
+import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition.IndexingRule;
+import org.apache.jackrabbit.oak.plugins.index.lucene.binary.BinaryTextExtractor;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.FacetsConfigProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.FunctionIndexProcessor;
-import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriter;
-import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
 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.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
@@ -48,242 +46,43 @@ import org.apache.lucene.document.LongFi
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
+import org.apache.lucene.facet.FacetsConfig;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
 import org.apache.lucene.util.BytesRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
+import static com.google.common.base.Preconditions.checkNotNull;
 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.FieldFactory.newAncestorsField;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newDepthField;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newFulltextField;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newPathField;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldFactory.newPropertyField;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.util.ConfigUtil.getPrimaryTypeName;
 
-/**
- * {@link IndexEditor} implementation that is responsible for keeping the
- * {@link LuceneIndex} up to date
- *
- * @see LuceneIndex
- */
-public class LuceneIndexEditor implements IndexEditor, Aggregate.AggregateRoot {
-
-    private static final Logger log =
-            LoggerFactory.getLogger(LuceneIndexEditor.class);
-
-    public static final String TEXT_EXTRACTION_ERROR = "TextExtractionError";
-
-    private final LuceneIndexEditorContext context;
-
-    /** Name of this node, or {@code null} for the root node. */
-    private final String name;
-
-    /** Parent editor or {@code null} if this is the root editor. */
-    private final LuceneIndexEditor parent;
-
-    /** Path of this editor, built lazily in {@link #getPath()}. */
-    private String path;
-
-    private boolean propertiesChanged = false;
-
-    private List<PropertyState> propertiesModified = Lists.newArrayList();
-
-    /**
-     * Flag indicating if the current tree being traversed has a deleted parent.
-     */
-    private final boolean isDeleted;
-
-    private IndexDefinition.IndexingRule indexingRule;
-
-    private List<Matcher> currentMatchers = Collections.emptyList();
-
-    private final MatcherState matcherState;
-
-    private final PathFilter.Result pathFilterResult;
-
-    LuceneIndexEditor(LuceneIndexEditorContext context) throws CommitFailedException {
-        this.parent = null;
-        this.name = null;
-        this.path = "/";
-        this.context = context;
-        this.isDeleted = false;
-        this.matcherState = MatcherState.NONE;
-        this.pathFilterResult = context.getDefinition().getPathFilter().filter(PathUtils.ROOT_PATH);
-    }
-
-    private LuceneIndexEditor(LuceneIndexEditor parent, String name,
-                              MatcherState matcherState,
-                              PathFilter.Result pathFilterResult,
-            boolean isDeleted) {
-        this.parent = parent;
-        this.name = name;
-        this.path = null;
-        this.context = parent.context;
-        this.isDeleted = isDeleted;
-        this.matcherState = matcherState;
-        this.pathFilterResult = pathFilterResult;
-    }
-
-    public String getPath() {
-        if (path == null) { // => parent != null
-            path = concat(parent.getPath(), name);
-        }
-        return path;
-    }
-
-    @Override
-    public void enter(NodeState before, NodeState after)
-            throws CommitFailedException {
-        if (EmptyNodeState.MISSING_NODE == before && parent == null){
-            context.enableReindexMode();
-        }
-
-        //Only check for indexing if the result is include.
-        //In case like TRAVERSE nothing needs to be indexed for those
-        //path
-        if (pathFilterResult == PathFilter.Result.INCLUDE) {
-            //For traversal in deleted sub tree before state has to be used
-            NodeState current = after.exists() ? after : before;
-            indexingRule = getDefinition().getApplicableIndexingRule(current);
-
-            if (indexingRule != null) {
-                currentMatchers = indexingRule.getAggregate().createMatchers(this);
-            }
-        }
-    }
-
-    @Override
-    public void leave(NodeState before, NodeState after)
-            throws CommitFailedException {
-        if (propertiesChanged || !before.exists()) {
-            String path = getPath();
-            if (addOrUpdate(path, after, before.exists())) {
-                long indexed = context.incIndexedNodes();
-                if (indexed % 1000 == 0) {
-                    log.debug("[{}] => Indexed {} nodes...", getIndexName(), indexed);
-                }
-            }
-        }
-
-        for (Matcher m : matcherState.affectedMatchers){
-            m.markRootDirty();
-        }
-
-        if (parent == null) {
-            try {
-                context.closeWriter();
-            } catch (IOException e) {
-                CommitFailedException ce = new CommitFailedException("Lucene", 4,
-                        "Failed to close the Lucene index " + context.getIndexingContext().getIndexPath(), e);
-                context.getIndexingContext().indexUpdateFailed(ce);
-                throw ce;
-            }
-            if (context.getIndexedNodes() > 0) {
-                log.debug("[{}] => Indexed {} nodes, done.", getIndexName(), context.getIndexedNodes());
-            }
-        }
-    }
-
-    @Override
-    public void propertyAdded(PropertyState after) {
-        markPropertyChanged(after.getName());
-        checkAggregates(after.getName());
-    }
-
-    @Override
-    public void propertyChanged(PropertyState before, PropertyState after) {
-        markPropertyChanged(before.getName());
-        propertiesModified.add(before);
-        checkAggregates(before.getName());
-    }
-
-    @Override
-    public void propertyDeleted(PropertyState before) {
-        markPropertyChanged(before.getName());
-        propertiesModified.add(before);
-        checkAggregates(before.getName());
+public class LuceneDocumentMaker {
+    private static final Logger log = LoggerFactory.getLogger(LuceneDocumentMaker.class);
+    private final BinaryTextExtractor textExtractor;
+    private final FacetsConfigProvider facetsConfigProvider;
+    private final IndexDefinition definition;
+    private final IndexingRule indexingRule;
+    private final IndexAugmentorFactory augmentorFactory;
+    private final String path;
+
+    public LuceneDocumentMaker(@Nullable BinaryTextExtractor textExtractor,
+                               @Nullable FacetsConfigProvider facetsConfigProvider,
+                               @Nullable IndexAugmentorFactory augmentorFactory,
+                               IndexDefinition definition, IndexingRule indexingRule,  String path) {
+        this.textExtractor = textExtractor;
+        this.facetsConfigProvider = facetsConfigProvider;
+        this.definition = checkNotNull(definition);
+        this.indexingRule = checkNotNull(indexingRule);
+        this.augmentorFactory = augmentorFactory;
+        this.path = checkNotNull(path);
     }
 
-    @Override
-    public Editor childNodeAdded(String name, NodeState after) {
-        PathFilter.Result filterResult = getPathFilterResult(name);
-        if (filterResult != PathFilter.Result.EXCLUDE) {
-            return new LuceneIndexEditor(this, name, getMatcherState(name, after), filterResult, false);
-        }
-        return null;
-    }
-
-    @Override
-    public Editor childNodeChanged(
-            String name, NodeState before, NodeState after) {
-        PathFilter.Result filterResult = getPathFilterResult(name);
-        if (filterResult != PathFilter.Result.EXCLUDE) {
-            return new LuceneIndexEditor(this, name, getMatcherState(name, after), filterResult, false);
-        }
-        return null;
-    }
-
-    @Override
-    public Editor childNodeDeleted(String name, NodeState before)
-            throws CommitFailedException {
-        PathFilter.Result filterResult = getPathFilterResult(name);
-        if (filterResult == PathFilter.Result.EXCLUDE) {
-            return null;
-        }
-
-        if (!isDeleted) {
-            // tree deletion is handled on the parent node
-            String path = concat(getPath(), name);
-            try {
-                LuceneIndexWriter writer = context.getWriter();
-                // Remove all index entries in the removed subtree
-                writer.deleteDocuments(path);
-                this.context.indexUpdate();
-            } catch (IOException e) {
-                CommitFailedException ce = new CommitFailedException("Lucene", 5, "Failed to remove the index entries of"
-                                + " the removed subtree " + path + "for index " + context.getIndexingContext().getIndexPath(), e);
-                context.getIndexingContext().indexUpdateFailed(ce);
-                throw ce;
-            }
-        }
-
-        MatcherState ms = getMatcherState(name, before);
-        if (!ms.isEmpty()){
-            return new LuceneIndexEditor(this, name, ms, filterResult, true);
-        }
-        return null; // no need to recurse down the removed subtree
-    }
-
-    LuceneIndexEditorContext getContext() {
-        return context;
-    }
-
-    private boolean addOrUpdate(String path, NodeState state, boolean isUpdate)
-            throws CommitFailedException {
-        try {
-            Document d = makeDocument(path, state, isUpdate);
-            if (d != null) {
-                if (log.isTraceEnabled()) {
-                    log.trace("[{}] Indexed document for {} is {}", getIndexName(), path, d);
-                }
-                context.indexUpdate();
-                context.getWriter().updateDocument(path, d);
-                return true;
-            }
-        } catch (IOException e) {
-            CommitFailedException ce = new CommitFailedException("Lucene", 3,
-                    "Failed to index the node " + path, e);
-            context.getIndexingContext().indexUpdateFailed(ce);
-            throw ce;
-        } catch (IllegalArgumentException ie) {
-            log.warn("Failed to index the node [{}]", path, ie);
-        }
-        return false;
-    }
-
-    private Document makeDocument(String path, NodeState state, boolean isUpdate) throws IOException {
-        if (!isIndexable()) {
-            return null;
-        }
-
+    public Document makeDocument(NodeState state, boolean isUpdate, List<PropertyState> propertiesModified) throws IOException {
         boolean facet = false;
 
         List<Field> fields = new ArrayList<Field>();
@@ -324,7 +123,7 @@ public class LuceneIndexEditor implement
 
         // Check if a node having a single property was modified/deleted
         if (!dirty) {
-            dirty = indexIfSinglePropertyRemoved();
+            dirty = indexIfSinglePropertyRemoved(propertiesModified);
         }
 
         if (isUpdate && !dirty) {
@@ -352,7 +151,7 @@ public class LuceneIndexEditor implement
             document.add(newFulltextField(name));
         }
 
-        if (getDefinition().evaluatePathRestrictions()){
+        if (definition.evaluatePathRestrictions()){
             document.add(newAncestorsField(PathUtils.getParentPath(path)));
             document.add(newDepthField(path));
         }
@@ -374,8 +173,8 @@ public class LuceneIndexEditor implement
             document.add(suggestField);
         }
 
-        if (facet) {
-            document = context.getFacetsConfig().build(document);
+        if (facet && isFacetingEnabled()) {
+            document = getFacetsConfig().build(document);
         }
 
         //TODO Boost at document level
@@ -385,7 +184,7 @@ public class LuceneIndexEditor implement
 
     private boolean addFacetFields(List<Field> fields, PropertyState property, String pname, PropertyDefinition pd) {
         String facetFieldName = FieldNames.createFacetFieldName(pname);
-        context.getFacetsConfig().setIndexFieldName(pname, facetFieldName);
+        getFacetsConfig().setIndexFieldName(pname, facetFieldName);
         int tag = property.getType().tag();
         int idxDefinedTag = pd.getType();
         // Try converting type to the defined type in the index definition
@@ -394,14 +193,14 @@ public class LuceneIndexEditor implement
                             + "path {}",
                     getIndexName(),
                     Type.fromTag(idxDefinedTag, false), property.toString(),
-                    Type.fromTag(tag, false), getPath());
+                    Type.fromTag(tag, false), path);
             tag = idxDefinedTag;
         }
 
         boolean fieldAdded = false;
         try {
             if (tag == Type.STRINGS.tag() && property.isArray()) {
-                context.getFacetsConfig().setMultiValued(pname, true);
+                getFacetsConfig().setMultiValued(pname, true);
                 Iterable<String> values = property.getValue(Type.STRINGS);
                 for (String value : values) {
                     if (value != null && value.length() > 0) {
@@ -421,7 +220,7 @@ public class LuceneIndexEditor implement
             log.warn("[{}] Ignoring facet property. Could not convert property {} of type {} to type {} for path {}",
                     getIndexName(), pname,
                     Type.fromTag(property.getType().tag(), false),
-                    Type.fromTag(tag, false), getPath(), e);
+                    Type.fromTag(tag, false), path, e);
         }
         return fieldAdded;
     }
@@ -466,7 +265,7 @@ public class LuceneIndexEditor implement
                     dirty = true;
                 }
             }
-            if (pd.facet) {
+            if (pd.facet && isFacetingEnabled()) {
                 dirty |= addFacetFields(fields, property, pname, pd);
             }
 
@@ -476,7 +275,7 @@ public class LuceneIndexEditor implement
     }
 
     private String constructAnalyzedPropertyName(String pname) {
-        if (context.getDefinition().getVersion().isAtLeast(IndexFormatVersion.V2)){
+        if (definition.getVersion().isAtLeast(IndexFormatVersion.V2)){
             return FieldNames.createAnalyzedFieldName(pname);
         }
         return pname;
@@ -515,7 +314,7 @@ public class LuceneIndexEditor implement
             log.warn(
                     "[{}] Ignoring ordered property {} of type {} for path {} as multivalued ordered property not supported",
                     getIndexName(), pname,
-                    Type.fromTag(property.getType().tag(), true), getPath());
+                    Type.fromTag(property.getType().tag(), true), path);
             return false;
         }
 
@@ -528,7 +327,7 @@ public class LuceneIndexEditor implement
                             + "path {}",
                     getIndexName(),
                     Type.fromTag(idxDefinedTag, false), property.toString(),
-                    Type.fromTag(tag, false), getPath());
+                    Type.fromTag(tag, false), path);
             tag = idxDefinedTag;
         }
 
@@ -562,7 +361,7 @@ public class LuceneIndexEditor implement
                     "[{}] Ignoring ordered property. Could not convert property {} of type {} to type {} for path {}",
                     getIndexName(), pname,
                     Type.fromTag(property.getType().tag(), false),
-                    Type.fromTag(tag, false), getPath(), e);
+                    Type.fromTag(tag, false), path, e);
         }
         return fieldAdded;
     }
@@ -573,24 +372,22 @@ public class LuceneIndexEditor implement
 
     private List<Field> newBinary(
             PropertyState property, NodeState state, String nodePath, String path) {
-        if (!context.isAsyncIndexing()){
+        if (textExtractor == null){
             //Skip text extraction for sync indexing
             return Collections.emptyList();
         }
 
-        return context.getTextExtractor().newBinary(property, state, nodePath, path);
+        return textExtractor.newBinary(property, state, nodePath, path);
     }
 
     private boolean augmentCustomFields(final String path, final List<Field> fields,
                                         final NodeState document) {
         boolean dirty = false;
 
-        IndexAugmentorFactory augmentorFactory = context.getAugmentorFactory();
         if (augmentorFactory != null) {
-            IndexDefinition defn = getDefinition();
             Iterable<Field> augmentedFields = augmentorFactory
                     .getIndexFieldProvider(indexingRule.getNodeTypeName())
-                    .getAugmentedFields(path, document, defn.getDefinitionNodeState());
+                    .getAugmentedFields(path, document, definition.getDefinitionNodeState());
 
             for (Field field : augmentedFields) {
                 fields.add(field);
@@ -639,18 +436,18 @@ public class LuceneIndexEditor implement
         }
         return fieldAdded;
     }
-    
+
     private static PropertyState calculateValue(String path, NodeState state, String[] functionCode) {
         try {
             return FunctionIndexProcessor.tryCalculateValue(path, state, functionCode);
         } catch (RuntimeException e) {
-            log.error("Failed to calculate function value for {} at {}", 
+            log.error("Failed to calculate function value for {} at {}",
                     Arrays.toString(functionCode), path, e);
             throw e;
         }
     }
 
-    private boolean indexIfSinglePropertyRemoved() {
+    private boolean indexIfSinglePropertyRemoved(List<PropertyState> propertiesModified) {
         boolean dirty = false;
         for (PropertyState ps : propertiesModified) {
             PropertyDefinition pd = indexingRule.getConfig(ps.getName());
@@ -708,33 +505,6 @@ public class LuceneIndexEditor implement
         return node;
     }
 
-    //~-------------------------------------------------------< Aggregate >
-
-    @Override
-    public void markDirty() {
-        propertiesChanged = true;
-    }
-
-    private MatcherState getMatcherState(String name, NodeState after) {
-        List<Matcher> matched = Lists.newArrayList();
-        List<Matcher> inherited = Lists.newArrayList();
-        for (Matcher m : Iterables.concat(matcherState.inherited, currentMatchers)) {
-            Matcher result = m.match(name, after);
-            if (result.getStatus() == Matcher.Status.MATCH_FOUND){
-                matched.add(result);
-            }
-
-            if (result.getStatus() != Matcher.Status.FAIL){
-                inherited.addAll(result.nextSet());
-            }
-        }
-
-        if (!matched.isEmpty() || !inherited.isEmpty()) {
-            return new MatcherState(matched, inherited);
-        }
-        return MatcherState.NONE;
-    }
-
     private boolean indexAggregates(final String path, final List<Field> fields,
                                     final NodeState state) {
         final AtomicBoolean dirtyFlag = new AtomicBoolean();
@@ -776,7 +546,7 @@ public class LuceneIndexEditor implement
         //rule for node being aggregated might be null if such nodes
         //are not indexed on there own. In such cases we rely in current
         //rule for some checks
-        IndexDefinition.IndexingRule ruleAggNode = context.getDefinition()
+        IndexDefinition.IndexingRule ruleAggNode = definition
                 .getApplicableIndexingRule(getPrimaryTypeName(result.nodeState));
         boolean dirty = false;
 
@@ -839,72 +609,18 @@ public class LuceneIndexEditor implement
         return dirty;
     }
 
-    /**
-     * Determines which all matchers are affected by this property change
-     *
-     * @param name modified property name
-     */
-    private void checkAggregates(String name) {
-        for (Matcher m : matcherState.matched) {
-            if (!matcherState.affectedMatchers.contains(m)
-                    && m.aggregatesProperty(name)) {
-                matcherState.affectedMatchers.add(m);
-            }
-        }
-    }
-
-    private static class MatcherState {
-        final static MatcherState NONE = new MatcherState(Collections.<Matcher>emptyList(),
-                Collections.<Matcher>emptyList());
-
-        final List<Matcher> matched;
-        final List<Matcher> inherited;
-        final Set<Matcher> affectedMatchers;
-
-        public MatcherState(List<Matcher> matched,
-                            List<Matcher> inherited){
-            this.matched = matched;
-            this.inherited = inherited;
-
-            //Affected matches would only be used when there are
-            //some matched matchers
-            if (matched.isEmpty()){
-                affectedMatchers = Collections.emptySet();
-            } else {
-                affectedMatchers = Sets.newIdentityHashSet();
-            }
-        }
-
-        public boolean isEmpty() {
-            return matched.isEmpty() && inherited.isEmpty();
-        }
-    }
-
-    private void markPropertyChanged(String name) {
-        if (isIndexable()
-                && !propertiesChanged
-                && indexingRule.isIndexed(name)) {
-            propertiesChanged = true;
-        }
-    }
-
-    private IndexDefinition getDefinition() {
-        return context.getDefinition();
-    }
-
-    private boolean isIndexable(){
-        return indexingRule != null;
+    private String getIndexName() {
+        return definition.getIndexName();
     }
 
-    private PathFilter.Result getPathFilterResult(String childNodeName) {
-        return context.getDefinition().getPathFilter().filter(concat(getPath(), childNodeName));
+    private boolean isFacetingEnabled(){
+        return facetsConfigProvider != null;
     }
 
-    private String getIndexName() {
-        return context.getDefinition().getIndexName();
+    private FacetsConfig getFacetsConfig(){
+        return facetsConfigProvider.getFacetsConfig();
     }
 
-
     /**
      * Extracts the local name of the current node ignoring any namespace prefix
      *
@@ -920,5 +636,4 @@ public class LuceneIndexEditor implement
         //cameCase file name to allow faster like search
         fields.add(new StringField(FieldNames.NODE_NAME, value, Field.Store.NO));
     }
-
 }

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=1777758&r1=1777757&r2=1777758&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 Sat Jan  7 10:10:07 2017
@@ -17,46 +17,28 @@
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 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.plugins.index.IndexEditor;
 import org.apache.jackrabbit.oak.plugins.index.PathFilter;
 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;
 import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
-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.lucene.document.Document;
-import org.apache.lucene.document.DoubleDocValuesField;
-import org.apache.lucene.document.DoubleField;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.LongField;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
-import org.apache.lucene.util.BytesRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 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.util.ConfigUtil.getPrimaryTypeName;
 
 /**
  * {@link IndexEditor} implementation that is responsible for keeping the
@@ -283,430 +265,9 @@ public class LuceneIndexEditor implement
         if (!isIndexable()) {
             return null;
         }
-
-        boolean facet = false;
-
-        List<Field> fields = new ArrayList<Field>();
-        boolean dirty = false;
-
-        //We 'intentionally' are indexing node names only on root state as we don't support indexing relative or
-        //regex for node name indexing
-        PropertyState nodenamePS =
-                new StringPropertyState(FieldNames.NODE_NAME, getName(path));
-        for (PropertyState property : Iterables.concat(state.getProperties(), Collections.singleton(nodenamePS))) {
-            String pname = property.getName();
-
-            if (!isVisible(pname) && !FieldNames.NODE_NAME.equals(pname)) {
-                continue;
-            }
-
-            PropertyDefinition pd = indexingRule.getConfig(pname);
-
-            if (pd == null || !pd.index){
-                continue;
-            }
-
-            if (pd.ordered) {
-                dirty |= addTypedOrderedFields(fields, property, pname, pd);
-            }
-
-            dirty |= indexProperty(path, fields, state, property, pname, pd);
-
-            facet |= pd.facet;
-        }
-
-        dirty |= indexAggregates(path, fields, state);
-        dirty |= indexNullCheckEnabledProps(path, fields, state);
-        dirty |= indexFunctionRestrictions(path, fields, state);
-        dirty |= indexNotNullCheckEnabledProps(path, fields, state);
-
-        dirty |= augmentCustomFields(path, fields, state);
-
-        // Check if a node having a single property was modified/deleted
-        if (!dirty) {
-            dirty = indexIfSinglePropertyRemoved();
-        }
-
-        if (isUpdate && !dirty) {
-            // updated the state but had no relevant changes
-            return null;
-        }
-
-        String name = getName(path);
-        if (indexingRule.isNodeNameIndexed()){
-            addNodeNameField(fields, name);
-            dirty = true;
-        }
-
-        //For property index no use making an empty document if
-        //none of the properties are indexed
-        if(!indexingRule.indexesAllNodesOfMatchingType() && !dirty){
-            return null;
-        }
-
-        Document document = new Document();
-        document.add(newPathField(path));
-
-
-        if (indexingRule.isFulltextEnabled()) {
-            document.add(newFulltextField(name));
-        }
-
-        if (getDefinition().evaluatePathRestrictions()){
-            document.add(newAncestorsField(PathUtils.getParentPath(path)));
-            document.add(newDepthField(path));
-        }
-
-        // because of LUCENE-5833 we have to merge the suggest fields into a single one
-        Field suggestField = null;
-        for (Field f : fields) {
-            if (FieldNames.SUGGEST.equals(f.name())) {
-                if (suggestField == null) {
-                    suggestField = FieldFactory.newSuggestField(f.stringValue());
-                } else {
-                    suggestField = FieldFactory.newSuggestField(suggestField.stringValue(), f.stringValue());
-                }
-            } else {
-                document.add(f);
-            }
-        }
-        if (suggestField != null) {
-            document.add(suggestField);
-        }
-
-        if (facet) {
-            document = context.getFacetsConfig().build(document);
-        }
-
-        //TODO Boost at document level
-
-        return document;
-    }
-
-    private boolean addFacetFields(List<Field> fields, PropertyState property, String pname, PropertyDefinition pd) {
-        String facetFieldName = FieldNames.createFacetFieldName(pname);
-        context.getFacetsConfig().setIndexFieldName(pname, facetFieldName);
-        int tag = property.getType().tag();
-        int idxDefinedTag = pd.getType();
-        // Try converting type to the defined type in the index definition
-        if (tag != idxDefinedTag) {
-            log.debug("[{}] Facet property defined with type {} differs from property {} with type {} in "
-                            + "path {}",
-                    getIndexName(),
-                    Type.fromTag(idxDefinedTag, false), property.toString(),
-                    Type.fromTag(tag, false), getPath());
-            tag = idxDefinedTag;
-        }
-
-        boolean fieldAdded = false;
-        try {
-            if (tag == Type.STRINGS.tag() && property.isArray()) {
-                context.getFacetsConfig().setMultiValued(pname, true);
-                Iterable<String> values = property.getValue(Type.STRINGS);
-                for (String value : values) {
-                    if (value != null && value.length() > 0) {
-                        fields.add(new SortedSetDocValuesFacetField(pname, value));
-                    }
-                }
-                fieldAdded = true;
-            } else if (tag == Type.STRING.tag()) {
-                String value = property.getValue(Type.STRING);
-                if (value.length() > 0) {
-                    fields.add(new SortedSetDocValuesFacetField(pname, value));
-                    fieldAdded = true;
-                }
-            }
-
-        } catch (Throwable e) {
-            log.warn("[{}] Ignoring facet property. Could not convert property {} of type {} to type {} for path {}",
-                    getIndexName(), pname,
-                    Type.fromTag(property.getType().tag(), false),
-                    Type.fromTag(tag, false), getPath(), e);
-        }
-        return fieldAdded;
-    }
-
-    private boolean indexProperty(String path,
-                                  List<Field> fields,
-                                  NodeState state,
-                                  PropertyState property,
-                                  String pname,
-                                  PropertyDefinition pd) {
-        boolean includeTypeForFullText = indexingRule.includePropertyType(property.getType().tag());
-
-        boolean dirty = false;
-        if (Type.BINARY.tag() == property.getType().tag()
-                && includeTypeForFullText) {
-            fields.addAll(newBinary(property, state, null, path + "@" + pname));
-            dirty = true;
-        } else {
-            if (pd.propertyIndex && pd.includePropertyType(property.getType().tag())) {
-                dirty |= addTypedFields(fields, property, pname);
-            }
-
-            if (pd.fulltextEnabled() && includeTypeForFullText) {
-                for (String value : property.getValue(Type.STRINGS)) {
-                    if (pd.analyzed && pd.includePropertyType(property.getType().tag())) {
-                        String analyzedPropName = constructAnalyzedPropertyName(pname);
-                        fields.add(newPropertyField(analyzedPropName, value, !pd.skipTokenization(pname), pd.stored));
-                    }
-
-                    if (pd.useInSuggest) {
-                        fields.add(FieldFactory.newSuggestField(value));
-                    }
-
-                    if (pd.useInSpellcheck) {
-                        fields.add(newPropertyField(FieldNames.SPELLCHECK, value, true, false));
-                    }
-
-                    if (pd.nodeScopeIndex) {
-                        Field field = newFulltextField(value);
-                        fields.add(field);
-                    }
-                    dirty = true;
-                }
-            }
-            if (pd.facet) {
-                dirty |= addFacetFields(fields, property, pname, pd);
-            }
-
-        }
-
-        return dirty;
-    }
-
-    private String constructAnalyzedPropertyName(String pname) {
-        if (context.getDefinition().getVersion().isAtLeast(IndexFormatVersion.V2)){
-            return FieldNames.createAnalyzedFieldName(pname);
-        }
-        return pname;
-    }
-
-    private boolean addTypedFields(List<Field> fields, PropertyState property, String pname) {
-        int tag = property.getType().tag();
-        boolean fieldAdded = false;
-        for (int i = 0; i < property.count(); i++) {
-            Field f;
-            if (tag == Type.LONG.tag()) {
-                f = new LongField(pname, property.getValue(Type.LONG, i), Field.Store.NO);
-            } else if (tag == Type.DATE.tag()) {
-                String date = property.getValue(Type.DATE, i);
-                f = new LongField(pname, FieldFactory.dateToLong(date), Field.Store.NO);
-            } else if (tag == Type.DOUBLE.tag()) {
-                f = new DoubleField(pname, property.getValue(Type.DOUBLE, i), Field.Store.NO);
-            } else if (tag == Type.BOOLEAN.tag()) {
-                f = new StringField(pname, property.getValue(Type.BOOLEAN, i).toString(), Field.Store.NO);
-            } else {
-                f = new StringField(pname, property.getValue(Type.STRING, i), Field.Store.NO);
-            }
-
-            fields.add(f);
-            fieldAdded = true;
-        }
-        return fieldAdded;
-    }
-
-    private boolean addTypedOrderedFields(List<Field> fields,
-                                          PropertyState property,
-                                          String pname,
-                                          PropertyDefinition pd) {
-        // Ignore and warn if property multi-valued as not supported
-        if (property.getType().isArray()) {
-            log.warn(
-                    "[{}] Ignoring ordered property {} of type {} for path {} as multivalued ordered property not supported",
-                    getIndexName(), pname,
-                    Type.fromTag(property.getType().tag(), true), getPath());
-            return false;
-        }
-
-        int tag = property.getType().tag();
-        int idxDefinedTag = pd.getType();
-        // Try converting type to the defined type in the index definition
-        if (tag != idxDefinedTag) {
-            log.debug(
-                    "[{}] Ordered property defined with type {} differs from property {} with type {} in "
-                            + "path {}",
-                    getIndexName(),
-                    Type.fromTag(idxDefinedTag, false), property.toString(),
-                    Type.fromTag(tag, false), getPath());
-            tag = idxDefinedTag;
-        }
-
-        String name = FieldNames.createDocValFieldName(pname);
-        boolean fieldAdded = false;
-        Field f = null;
-        try {
-            if (tag == Type.LONG.tag()) {
-                //TODO Distinguish fields which need to be used for search and for sort
-                //If a field is only used for Sort then it can be stored with less precision
-                f = new NumericDocValuesField(name, property.getValue(Type.LONG));
-            } else if (tag == Type.DATE.tag()) {
-                String date = property.getValue(Type.DATE);
-                f = new NumericDocValuesField(name, FieldFactory.dateToLong(date));
-            } else if (tag == Type.DOUBLE.tag()) {
-                f = new DoubleDocValuesField(name, property.getValue(Type.DOUBLE));
-            } else if (tag == Type.BOOLEAN.tag()) {
-                f = new SortedDocValuesField(name,
-                    new BytesRef(property.getValue(Type.BOOLEAN).toString()));
-            } else if (tag == Type.STRING.tag()) {
-                f = new SortedDocValuesField(name,
-                    new BytesRef(property.getValue(Type.STRING)));
-            }
-
-            if (f != null) {
-                fields.add(f);
-                fieldAdded = true;
-            }
-        } catch (Exception e) {
-            log.warn(
-                    "[{}] Ignoring ordered property. Could not convert property {} of type {} to type {} for path {}",
-                    getIndexName(), pname,
-                    Type.fromTag(property.getType().tag(), false),
-                    Type.fromTag(tag, false), getPath(), e);
-        }
-        return fieldAdded;
-    }
-
-    private static boolean isVisible(String name) {
-        return name.charAt(0) != ':';
-    }
-
-    private List<Field> newBinary(
-            PropertyState property, NodeState state, String nodePath, String path) {
-        if (!context.isAsyncIndexing()){
-            //Skip text extraction for sync indexing
-            return Collections.emptyList();
-        }
-
-        return context.getTextExtractor().newBinary(property, state, nodePath, path);
-    }
-
-    private boolean augmentCustomFields(final String path, final List<Field> fields,
-                                        final NodeState document) {
-        boolean dirty = false;
-
-        IndexAugmentorFactory augmentorFactory = context.getAugmentorFactory();
-        if (augmentorFactory != null) {
-            IndexDefinition defn = getDefinition();
-            Iterable<Field> augmentedFields = augmentorFactory
-                    .getIndexFieldProvider(indexingRule.getNodeTypeName())
-                    .getAugmentedFields(path, document, defn.getDefinitionNodeState());
-
-            for (Field field : augmentedFields) {
-                fields.add(field);
-                dirty = true;
-            }
-        }
-
-        return dirty;
-    }
-
-    //~-------------------------------------------------------< NullCheck Support >
-
-    private boolean indexNotNullCheckEnabledProps(String path, List<Field> fields, NodeState state) {
-        boolean fieldAdded = false;
-        for (PropertyDefinition pd : indexingRule.getNotNullCheckEnabledProperties()) {
-            if (isPropertyNotNull(state, pd)) {
-                fields.add(new StringField(FieldNames.NOT_NULL_PROPS, pd.name, Field.Store.NO));
-                fieldAdded = true;
-            }
-        }
-        return fieldAdded;
-    }
-
-    private boolean indexNullCheckEnabledProps(String path, List<Field> fields, NodeState state) {
-        boolean fieldAdded = false;
-        for (PropertyDefinition pd : indexingRule.getNullCheckEnabledProperties()) {
-            if (isPropertyNull(state, pd)) {
-                fields.add(new StringField(FieldNames.NULL_PROPS, pd.name, Field.Store.NO));
-                fieldAdded = true;
-            }
-        }
-        return fieldAdded;
-    }
-
-    private boolean indexFunctionRestrictions(String path, List<Field> fields, NodeState state) {
-        boolean fieldAdded = false;
-        for (PropertyDefinition pd : indexingRule.getFunctionRestrictions()) {
-            PropertyState functionValue = calculateValue(path, state, pd.functionCode);
-            if (functionValue != null) {
-                if (pd.ordered) {
-                    addTypedOrderedFields(fields, functionValue, pd.function, pd);
-                }
-                addTypedFields(fields, functionValue, pd.function);
-                fieldAdded = true;
-            }
-        }
-        return fieldAdded;
-    }
-    
-    private static PropertyState calculateValue(String path, NodeState state, String[] functionCode) {
-        try {
-            return FunctionIndexProcessor.tryCalculateValue(path, state, functionCode);
-        } catch (RuntimeException e) {
-            log.error("Failed to calculate function value for {} at {}", 
-                    Arrays.toString(functionCode), path, e);
-            throw e;
-        }
-    }
-
-    private boolean indexIfSinglePropertyRemoved() {
-        boolean dirty = false;
-        for (PropertyState ps : propertiesModified) {
-            PropertyDefinition pd = indexingRule.getConfig(ps.getName());
-            if (pd != null
-                    && pd.index
-                    && (pd.includePropertyType(ps.getType().tag())
-                            || indexingRule.includePropertyType(ps.getType().tag()))) {
-                dirty = true;
-                break;
-            }
-        }
-        return dirty;
-    }
-
-    /**
-     * Determine if the property as defined by PropertyDefinition exists or not.
-     *
-     * <p>For relative property if the intermediate nodes do not exist then property is
-     * <bold>not</bold> considered to be null</p>
-     *
-     * @return true if the property does not exist
-     */
-    private boolean isPropertyNull(NodeState state, PropertyDefinition pd){
-        NodeState propertyNode = getPropertyNode(state, pd);
-        if (!propertyNode.exists()){
-            return false;
-        }
-        return !propertyNode.hasProperty(pd.nonRelativeName);
-    }
-
-    /**
-     * Determine if the property as defined by PropertyDefinition exists or not.
-     *
-     * <p>For relative property if the intermediate nodes do not exist then property is
-     * considered to be null</p>
-     *
-     * @return true if the property exists
-     */
-    private boolean isPropertyNotNull(NodeState state, PropertyDefinition pd){
-        NodeState propertyNode = getPropertyNode(state, pd);
-        if (!propertyNode.exists()){
-            return false;
-        }
-        return propertyNode.hasProperty(pd.nonRelativeName);
+        return context.newDocumentMaker(indexingRule, path).makeDocument(state, isUpdate, propertiesModified);
     }
 
-    private static NodeState getPropertyNode(NodeState nodeState, PropertyDefinition pd) {
-        if (!pd.relative){
-            return nodeState;
-        }
-        NodeState node = nodeState;
-        for (String name : pd.ancestors) {
-            node = node.getChildNode(name);
-        }
-        return node;
-    }
 
     //~-------------------------------------------------------< Aggregate >
 
@@ -735,109 +296,6 @@ public class LuceneIndexEditor implement
         return MatcherState.NONE;
     }
 
-    private boolean indexAggregates(final String path, final List<Field> fields,
-                                    final NodeState state) {
-        final AtomicBoolean dirtyFlag = new AtomicBoolean();
-        indexingRule.getAggregate().collectAggregates(state, new Aggregate.ResultCollector() {
-            @Override
-            public void onResult(Aggregate.NodeIncludeResult result) {
-                boolean dirty = indexAggregatedNode(path, fields, result);
-                if (dirty) {
-                    dirtyFlag.set(true);
-                }
-            }
-
-            @Override
-            public void onResult(Aggregate.PropertyIncludeResult result) {
-                boolean dirty = false;
-                if (result.pd.ordered) {
-                    dirty |= addTypedOrderedFields(fields, result.propertyState,
-                            result.propertyPath, result.pd);
-                }
-                dirty |= indexProperty(path, fields, state, result.propertyState,
-                        result.propertyPath, result.pd);
-
-                if (dirty) {
-                    dirtyFlag.set(true);
-                }
-            }
-        });
-        return dirtyFlag.get();
-    }
-    /**
-     * Create the fulltext field from the aggregated nodes. If result is for aggregate for a relative node
-     * include then
-     * @param path current node path
-     * @param fields indexed fields
-     * @param result aggregate result
-     * @return true if a field was created for passed node result
-     */
-    private boolean indexAggregatedNode(String path, List<Field> fields, Aggregate.NodeIncludeResult result) {
-        //rule for node being aggregated might be null if such nodes
-        //are not indexed on there own. In such cases we rely in current
-        //rule for some checks
-        IndexDefinition.IndexingRule ruleAggNode = context.getDefinition()
-                .getApplicableIndexingRule(getPrimaryTypeName(result.nodeState));
-        boolean dirty = false;
-
-        for (PropertyState property : result.nodeState.getProperties()){
-            String pname = property.getName();
-            String propertyPath = PathUtils.concat(result.nodePath, pname);
-
-            if (!isVisible(pname)) {
-                continue;
-            }
-
-            //Check if type is indexed
-            int type = property.getType().tag();
-            if (ruleAggNode != null ) {
-                if (!ruleAggNode.includePropertyType(type)) {
-                    continue;
-                }
-            } else if (!indexingRule.includePropertyType(type)){
-                continue;
-            }
-
-            //Check if any explicit property defn is defined via relative path
-            // and is marked to exclude this property from being indexed. We exclude
-            //it from aggregation if
-            // 1. Its not to be indexed i.e. index=false
-            // 2. Its explicitly excluded from aggregation i.e. excludeFromAggregation=true
-            PropertyDefinition pdForRootNode = indexingRule.getConfig(propertyPath);
-            if (pdForRootNode != null && (!pdForRootNode.index || pdForRootNode.excludeFromAggregate)) {
-                continue;
-            }
-
-            if (Type.BINARY == property.getType()) {
-                String aggreagtedNodePath = PathUtils.concat(path, result.nodePath);
-                //Here the fulltext is being created for aggregate root hence nodePath passed
-                //should be null
-                String nodePath = result.isRelativeNode() ? result.rootIncludePath : null;
-                fields.addAll(newBinary(property, result.nodeState, nodePath, aggreagtedNodePath + "@" + pname));
-                dirty = true;
-            } else {
-                PropertyDefinition pd = null;
-                if (ruleAggNode != null){
-                    pd = ruleAggNode.getConfig(pname);
-                }
-
-                if (pd != null && !pd.nodeScopeIndex){
-                    continue;
-                }
-
-                for (String value : property.getValue(Type.STRINGS)) {
-                    Field field = result.isRelativeNode() ?
-                            newFulltextField(result.rootIncludePath, value) : newFulltextField(value) ;
-                    if (pd != null) {
-                        field.setBoost(pd.boost);
-                    }
-                    fields.add(field);
-                    dirty = true;
-                }
-            }
-        }
-        return dirty;
-    }
 
     /**
      * Determines which all matchers are affected by this property change
@@ -903,22 +361,4 @@ public class LuceneIndexEditor implement
     private String getIndexName() {
         return context.getDefinition().getIndexName();
     }
-
-
-    /**
-     * Extracts the local name of the current node ignoring any namespace prefix
-     *
-     * @param name node name
-     */
-    private static void addNodeNameField(List<Field> fields, String name) {
-        //TODO Need to check if it covers all cases
-        int colon = name.indexOf(':');
-        String value = colon < 0 ? name : name.substring(colon + 1);
-
-        //For now just add a single term. Later we can look into using different analyzer
-        //to analyze the node name and add multiple terms. Like add multiple terms for a
-        //cameCase file name to allow faster like search
-        fields.add(new StringField(FieldNames.NODE_NAME, value, Field.Store.NO));
-    }
-
 }

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=1777758&r1=1777757&r2=1777758&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 Sat Jan  7 10:10:07 2017
@@ -27,6 +27,7 @@ import org.apache.jackrabbit.oak.plugins
 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.util.FacetsConfigProvider;
 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;
@@ -43,7 +44,7 @@ import static com.google.common.base.Pre
 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;
 
-public class LuceneIndexEditorContext {
+public class LuceneIndexEditorContext implements FacetsConfigProvider{
 
     private static final Logger log = LoggerFactory
             .getLogger(LuceneIndexEditorContext.class);
@@ -188,7 +189,7 @@ public class LuceneIndexEditorContext {
         return indexedNodes;
     }
 
-    public boolean isAsyncIndexing() {
+    private boolean isAsyncIndexing() {
         return asyncIndexing;
     }
 
@@ -204,25 +205,29 @@ public class LuceneIndexEditorContext {
         return definition;
     }
 
-    FacetsConfig getFacetsConfig() {
+    LuceneDocumentMaker newDocumentMaker(IndexDefinition.IndexingRule rule, String path){
+        //Faceting is only enabled for async mode
+        FacetsConfigProvider facetsConfigProvider = isAsyncIndexing() ? this : null;
+        return new LuceneDocumentMaker(getTextExtractor(), facetsConfigProvider, augmentorFactory,
+                definition, rule, path);
+    }
+
+    @Override
+    public FacetsConfig getFacetsConfig() {
         if (facetsConfig == null){
             facetsConfig = FacetHelper.getFacetsConfig(definitionBuilder);
         }
         return facetsConfig;
     }
 
-    BinaryTextExtractor getTextExtractor(){
-        if (textExtractor == null){
+    private BinaryTextExtractor getTextExtractor(){
+        if (textExtractor == null && isAsyncIndexing()){
             //Create lazily to ensure that if its reindex case then update definition is picked
             textExtractor = new BinaryTextExtractor(extractedTextCache, definition, reindex);
         }
         return textExtractor;
     }
 
-    IndexAugmentorFactory getAugmentorFactory() {
-        return augmentorFactory;
-    }
-
     public boolean isReindex() {
         return reindex;
     }

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FacetsConfigProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FacetsConfigProvider.java?rev=1777758&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FacetsConfigProvider.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FacetsConfigProvider.java Sat Jan  7 10:10:07 2017
@@ -0,0 +1,31 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.facet.FacetsConfig;
+
+/**
+ * Ensures that FacetsConfig is lazily created only when facets are to be created
+ * and only for those indexes where facets are configured
+ */
+public interface FacetsConfigProvider {
+
+    FacetsConfig getFacetsConfig();
+}

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