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 al...@apache.org on 2013/05/06 12:42:37 UTC

svn commit: r1479514 - in /jackrabbit/oak/trunk: oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/ oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/ oak-lu...

Author: alexparvulescu
Date: Mon May  6 10:41:23 2013
New Revision: 1479514

URL: http://svn.apache.org/r1479514
Log:
OAK-811 Refactor Lucene index impl

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java   (contents, props changed)
      - copied, changed from r1479477, jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java   (with props)
Removed:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUtils.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexEditor.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/QueryIndex.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.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/LuceneIndexProvider.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexUpdate.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneInitializerHelper.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUtils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUtils.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUtils.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexUtils.java Mon May  6 10:41:23 2013
@@ -24,6 +24,7 @@ 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.property.PropertyIndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
 import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
@@ -177,10 +178,19 @@ public class IndexUtils {
 
     public static boolean isIndexNodeType(NodeState state) {
         PropertyState ps = state.getProperty(JCR_PRIMARYTYPE);
-        return ps != null && !ps.isArray()
+        return ps != null
                 && ps.getValue(STRING).equals(INDEX_DEFINITIONS_NODE_TYPE);
     }
 
+    public static boolean isIndexNodeType(NodeState state, String typeIn) {
+        if (!isIndexNodeType(state)) {
+            return false;
+        }
+        PropertyState type = state.getProperty(TYPE_PROPERTY_NAME);
+        return type != null && !type.isArray()
+                && type.getValue(Type.STRING).equals(typeIn);
+    }
+
     public static String getString(NodeBuilder builder, String name) {
         PropertyState property = builder.getProperty(name);
         if (property != null && property.getType() == STRING) {
@@ -211,4 +221,12 @@ public class IndexUtils {
                 && property.getValue(BOOLEAN);
     }
 
+    public static NodeBuilder getChildOrNull(NodeBuilder node, String name) {
+        if (node != null && node.hasChildNode(name)) {
+            return node.child(name);
+        } else {
+            return null;
+        }
+    }
+
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexEditor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexEditor.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexEditor.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexEditor.java Mon May  6 10:41:23 2013
@@ -20,14 +20,11 @@ import static com.google.common.collect.
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Sets.newHashSet;
 import static org.apache.jackrabbit.JcrConstants.JCR_ISMIXIN;
-import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
 import static org.apache.jackrabbit.JcrConstants.JCR_SYSTEM;
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.DECLARING_NODE_TYPES;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NODE_TYPE;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.PROPERTY_NAMES;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexEditorProvider.TYPE;
 import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
 import static org.apache.jackrabbit.oak.plugins.nodetype.NodeTypeConstants.JCR_NODE_TYPES;
@@ -46,6 +43,7 @@ import org.apache.jackrabbit.oak.api.Com
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.plugins.index.IndexEditor;
+import org.apache.jackrabbit.oak.plugins.index.IndexUtils;
 import org.apache.jackrabbit.oak.plugins.index.property.strategy.ContentMirrorStoreStrategy;
 import org.apache.jackrabbit.oak.plugins.index.property.strategy.IndexStoreStrategy;
 import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
@@ -109,7 +107,7 @@ class PropertyIndexEditor implements Ind
     }
 
     private PropertyIndexEditor(PropertyIndexEditor parent, String nodeName) {
-        this(parent, getChildNode(parent.node, nodeName), nodeName, null);
+        this(parent, IndexUtils.getChildOrNull(parent.node, nodeName), nodeName, null);
     }
 
     private PropertyIndexEditor(PropertyIndexEditor parent, NodeBuilder node,
@@ -134,14 +132,6 @@ class PropertyIndexEditor implements Ind
         }
     }
 
-    private static NodeBuilder getChildNode(NodeBuilder node, String name) {
-        if (node != null && node.hasChildNode(name)) {
-            return node.child(name);
-        } else {
-            return null;
-        }
-    }
-
     public String getPath() {
         // build the path lazily
         if (path == null) {
@@ -221,19 +211,6 @@ class PropertyIndexEditor implements Ind
         }
     }
 
-    private static boolean isIndexNode(NodeState node) {
-        PropertyState ps = node.getProperty(JCR_PRIMARYTYPE);
-        boolean isNodeType = ps != null && !ps.isArray()
-                && ps.getValue(Type.STRING).equals(INDEX_DEFINITIONS_NODE_TYPE);
-        if (!isNodeType) {
-            return false;
-        }
-        PropertyState type = node.getProperty(TYPE_PROPERTY_NAME);
-        boolean isIndexType = type != null && !type.isArray()
-                && type.getValue(Type.STRING).equals(TYPE);
-        return isIndexType;
-    }
-
     @Override
     public void enter(NodeState before, NodeState after)
             throws CommitFailedException {
@@ -241,7 +218,7 @@ class PropertyIndexEditor implements Ind
             NodeState index = after.getChildNode(INDEX_DEFINITIONS_NAME);
             for (String indexName : index.getChildNodeNames()) {
                 NodeState child = index.getChildNode(indexName);
-                if (isIndexNode(child)) {
+                if (IndexUtils.isIndexNodeType(child, TYPE)) {
                     addIndexes(child, indexName);
                 }
             }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/QueryIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/QueryIndex.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/QueryIndex.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/spi/query/QueryIndex.java Mon May  6 10:41:23 2013
@@ -18,8 +18,6 @@
  */
 package org.apache.jackrabbit.oak.spi.query;
 
-import java.util.List;
-
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 /**

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java Mon May  6 10:41:23 2013
@@ -16,6 +16,19 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
+import static org.apache.jackrabbit.JcrConstants.JCR_MIXINTYPES;
+import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.getString;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH_SELECTOR;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INDEX_DATA_CHILD_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.TYPE_LUCENE;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newPathTerm;
+import static org.apache.jackrabbit.oak.query.Query.JCR_PATH;
+import static org.apache.lucene.search.BooleanClause.Occur.SHOULD;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -24,13 +37,12 @@ import java.util.List;
 
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.PathUtils;
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
 import org.apache.jackrabbit.oak.spi.query.Cursor;
 import org.apache.jackrabbit.oak.spi.query.Cursors;
 import org.apache.jackrabbit.oak.spi.query.Filter;
 import org.apache.jackrabbit.oak.spi.query.Filter.PropertyRestriction;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
-import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.apache.jackrabbit.oak.spi.state.ReadOnlyBuilder;
 import org.apache.lucene.index.DirectoryReader;
@@ -52,15 +64,6 @@ import org.apache.lucene.store.Directory
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.jackrabbit.JcrConstants.JCR_MIXINTYPES;
-import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
-import static org.apache.jackrabbit.oak.commons.PathUtils.elements;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH_SELECTOR;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newPathTerm;
-import static org.apache.jackrabbit.oak.query.Query.JCR_PATH;
-import static org.apache.lucene.search.BooleanClause.Occur.SHOULD;
-
 /**
  * Provides a QueryIndex that does lookups against a Lucene-based index
  * 
@@ -93,26 +96,47 @@ import static org.apache.lucene.search.B
  * @see QueryIndex
  * 
  */
-public class LuceneIndex implements QueryIndex, LuceneIndexConstants {
+public class LuceneIndex implements QueryIndex {
 
     private static final Logger LOG = LoggerFactory
             .getLogger(LuceneIndex.class);
 
-    private final IndexDefinition index;
-
-    public LuceneIndex(IndexDefinition indexDefinition) {
-        this.index = indexDefinition;
+    public LuceneIndex() {
     }
 
     @Override
     public String getIndexName() {
-        return index.getName();
+        return "lucene";
     }
 
     @Override
     public double getCost(Filter filter, NodeState root) {
         // TODO: proper cost calculation
-        return 1.0;
+        NodeState index = getIndexDataNode(root);
+        if (index == null) {
+            // unusable index
+            return Double.POSITIVE_INFINITY;
+        }
+        if (!filter.getFulltextConditions().isEmpty()) {
+            return 0.5;
+        }
+        // no fulltext, don't use this index
+        return Double.POSITIVE_INFINITY;
+    }
+
+    private static NodeState getIndexDataNode(NodeState node) {
+        NodeState state = node.getChildNode(INDEX_DEFINITIONS_NAME);
+        for (ChildNodeEntry entry : state.getChildNodeEntries()) {
+            NodeState ns = entry.getNodeState();
+            if (TYPE_LUCENE.equals(getString(ns, TYPE_PROPERTY_NAME))) {
+                if (ns.hasChildNode(INDEX_DATA_CHILD_NAME)) {
+                    return ns.getChildNode(INDEX_DATA_CHILD_NAME);
+                }
+                // unusable index (not initialized yet)
+                return null;
+            }
+        }
+        return null;
     }
 
     @Override
@@ -122,18 +146,12 @@ public class LuceneIndex implements Quer
 
     @Override
     public Cursor query(Filter filter, NodeState root) {
-
-        NodeBuilder builder = new ReadOnlyBuilder(root);
-        for (String name : elements(index.getPath())) {
-            builder = builder.child(name);
-        }
-        if (!builder.hasChildNode(INDEX_DATA_CHILD_NAME)) {
-            // index not initialized yet
+        NodeState index = getIndexDataNode(root);
+        if (index == null) {
             return Cursors.newPathCursor(Collections.<String> emptySet());
         }
-        builder = builder.child(INDEX_DATA_CHILD_NAME);
-
-        Directory directory = new ReadOnlyOakDirectory(builder);
+        Directory directory = new ReadOnlyOakDirectory(new ReadOnlyBuilder(
+                index));
         long s = System.currentTimeMillis();
 
         try {
@@ -318,9 +336,4 @@ public class LuceneIndex implements Quer
         qs.add(bq);
     }
 
-    @Override
-    public String toString() {
-        return "LuceneIndex [index=" + index + "]";
-    }
-
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java Mon May  6 10:41:23 2013
@@ -16,12 +16,11 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
-import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.util.Version;
 
-public interface LuceneIndexConstants extends IndexConstants {
+public interface LuceneIndexConstants {
 
     String TYPE_LUCENE = "lucene";
 
@@ -31,4 +30,9 @@ public interface LuceneIndexConstants ex
 
     Analyzer ANALYZER = new StandardAnalyzer(VERSION);
 
+    /**
+     * include only certain property types in the index
+     */
+    String INCLUDE_PROPERTY_TYPES = "includePropertyTypes";
+
 }

Copied: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java (from r1479477, jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java)
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java?p2=jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java&p1=jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java&r1=1479477&r2=1479514&rev=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java Mon May  6 10:41:23 2013
@@ -16,13 +16,14 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
-import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NODE_TYPE;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.getString;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.getChildOrNull;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.TYPE_LUCENE;
 import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
+import static org.apache.jackrabbit.oak.spi.state.NodeStateUtils.isHidden;
 
 import java.io.Closeable;
 import java.io.IOException;
@@ -31,12 +32,10 @@ import java.util.Map;
 
 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.IndexEditor;
 import org.apache.jackrabbit.oak.spi.commit.Editor;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
-import org.apache.jackrabbit.oak.spi.state.NodeStateUtils;
 import org.apache.jackrabbit.oak.spi.state.ReadOnlyBuilder;
 import org.apache.tika.config.TikaConfig;
 import org.apache.tika.parser.AutoDetectParser;
@@ -49,9 +48,9 @@ import org.apache.tika.parser.Parser;
  * @see LuceneIndex
  * 
  */
-public class LuceneIndexDiff implements IndexEditor, Closeable {
+public class LuceneIndexEditor implements IndexEditor, Closeable {
 
-    private final LuceneIndexDiff parent;
+    private final LuceneIndexEditor parent;
 
     private final NodeBuilder node;
 
@@ -69,7 +68,7 @@ public class LuceneIndexDiff implements 
     private final Parser parser = new AutoDetectParser(
             TikaConfig.getDefaultConfig());
 
-    private LuceneIndexDiff(LuceneIndexDiff parent, NodeBuilder node,
+    private LuceneIndexEditor(LuceneIndexEditor parent, NodeBuilder node,
             String name, String path, Map<String, LuceneIndexUpdate> updates,
             boolean isRoot) {
         this.parent = parent;
@@ -80,24 +79,16 @@ public class LuceneIndexDiff implements 
         this.isRoot = isRoot;
     }
 
-    private LuceneIndexDiff(LuceneIndexDiff parent, String name) {
-        this(parent, getChildNode(parent.node, name), name, null,
+    private LuceneIndexEditor(LuceneIndexEditor parent, String name) {
+        this(parent, getChildOrNull(parent.node, name), name, null,
                 parent.updates, false);
     }
 
-    public LuceneIndexDiff(NodeBuilder root) {
+    public LuceneIndexEditor(NodeBuilder root) {
         this(null, root, null, "/", new HashMap<String, LuceneIndexUpdate>(),
                 true);
     }
 
-    private static NodeBuilder getChildNode(NodeBuilder node, String name) {
-        if (node != null && node.hasChildNode(name)) {
-            return node.child(name);
-        } else {
-            return null;
-        }
-    }
-
     public String getPath() {
         if (path == null) { // => parent != null
             path = concat(parent.getPath(), name);
@@ -106,16 +97,7 @@ public class LuceneIndexDiff implements 
     }
 
     private static boolean isIndexNode(NodeBuilder node) {
-        PropertyState ps = node.getProperty(JCR_PRIMARYTYPE);
-        boolean isNodeType = ps != null && !ps.isArray()
-                && ps.getValue(Type.STRING).equals(INDEX_DEFINITIONS_NODE_TYPE);
-        if (!isNodeType) {
-            return false;
-        }
-        PropertyState type = node.getProperty(TYPE_PROPERTY_NAME);
-        boolean isIndexType = type != null && !type.isArray()
-                && type.getValue(Type.STRING).equals(TYPE_LUCENE);
-        return isIndexType;
+        return TYPE_LUCENE.equals(getString(node, TYPE_PROPERTY_NAME));
     }
 
     @Override
@@ -131,7 +113,7 @@ public class LuceneIndexDiff implements 
                 }
             }
         }
-        if (node != null && name != null && !NodeStateUtils.isHidden(name)) {
+        if (node != null && name != null && !isHidden(name)) {
             for (LuceneIndexUpdate update : updates.values()) {
                 update.insert(getPath(), node);
             }
@@ -147,24 +129,27 @@ public class LuceneIndexDiff implements 
         for (LuceneIndexUpdate update : updates.values()) {
             update.apply();
         }
+        updates.clear();
     }
 
     @Override
-    public void propertyAdded(PropertyState after) {
+    public void propertyAdded(PropertyState after) throws CommitFailedException {
         for (LuceneIndexUpdate update : updates.values()) {
             update.insert(getPath(), node);
         }
     }
 
     @Override
-    public void propertyChanged(PropertyState before, PropertyState after) {
+    public void propertyChanged(PropertyState before, PropertyState after)
+            throws CommitFailedException {
         for (LuceneIndexUpdate update : updates.values()) {
             update.insert(getPath(), node);
         }
     }
 
     @Override
-    public void propertyDeleted(PropertyState before) {
+    public void propertyDeleted(PropertyState before)
+            throws CommitFailedException {
         for (LuceneIndexUpdate update : updates.values()) {
             update.insert(getPath(), node);
         }
@@ -173,7 +158,7 @@ public class LuceneIndexDiff implements 
     @Override
     public Editor childNodeAdded(String name, NodeState after)
             throws CommitFailedException {
-        if (NodeStateUtils.isHidden(name)) {
+        if (isHidden(name)) {
             return null;
         }
         for (LuceneIndexUpdate update : updates.values()) {
@@ -185,16 +170,16 @@ public class LuceneIndexDiff implements 
     @Override
     public Editor childNodeChanged(String name, NodeState before,
             NodeState after) throws CommitFailedException {
-        if (NodeStateUtils.isHidden(name)) {
+        if (isHidden(name)) {
             return null;
         }
-        return new LuceneIndexDiff(this, name);
+        return new LuceneIndexEditor(this, name);
     }
 
     @Override
     public Editor childNodeDeleted(String name, NodeState before)
             throws CommitFailedException {
-        if (NodeStateUtils.isHidden(name)) {
+        if (isHidden(name)) {
             return null;
         }
         for (LuceneIndexUpdate update : updates.values()) {

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

Propchange: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

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=1479514&r1=1479513&r2=1479514&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 Mon May  6 10:41:23 2013
@@ -28,7 +28,7 @@ import org.apache.jackrabbit.oak.spi.sta
 /**
  * Service that provides Lucene based {@link IndexEditor}s
  * 
- * @see LuceneIndexDiff
+ * @see LuceneIndexEditor
  * @see IndexEditorProvider
  * 
  */
@@ -39,7 +39,7 @@ public class LuceneIndexEditorProvider i
     @Override
     public Editor getIndexEditor(String type, NodeBuilder builder) {
         if (TYPE_LUCENE.equals(type)) {
-            return new LuceneIndexDiff(builder);
+            return new LuceneIndexEditor(builder);
         }
         return null;
     }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java Mon May  6 10:41:23 2013
@@ -16,21 +16,17 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
-import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.buildIndexDefinitions;
-
-import java.util.ArrayList;
 import java.util.List;
 
 import javax.annotation.Nonnull;
 
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Service;
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
 
 /**
  * A provider for Lucene indexes.
@@ -43,21 +39,15 @@ import org.slf4j.LoggerFactory;
 public class LuceneIndexProvider implements QueryIndexProvider,
         LuceneIndexConstants {
 
-    private static final Logger LOG = LoggerFactory
-            .getLogger(LuceneIndexProvider.class);
-
     @Override @Nonnull
     public List<QueryIndex> getQueryIndexes(NodeState nodeState) {
-        List<QueryIndex> tempIndexes = new ArrayList<QueryIndex>();
-        for (IndexDefinition child : buildIndexDefinitions(nodeState, "/",
-                TYPE_LUCENE)) {
-            LOG.debug("found a lucene index definition {}", child);
-            tempIndexes.add(newLuceneIndex(child));
-        }
-        return tempIndexes;
+        return ImmutableList.<QueryIndex>of(newLuceneIndex());
     }
 
-    protected LuceneIndex newLuceneIndex(IndexDefinition child) {
-        return new LuceneIndex(child);
+    /**
+     * testing purposes
+     */
+    protected LuceneIndex newLuceneIndex() {
+        return new LuceneIndex();
     }
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexUpdate.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexUpdate.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexUpdate.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexUpdate.java Mon May  6 10:41:23 2013
@@ -16,6 +16,8 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.getString;
 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.TermFactory.newPathTerm;
@@ -23,13 +25,10 @@ import static org.apache.jackrabbit.oak.
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.Map;
+import java.util.HashSet;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.TreeSet;
 
-import javax.jcr.PropertyType;
-
 import org.apache.jackrabbit.JcrConstants;
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
@@ -50,8 +49,6 @@ import org.apache.tika.sax.WriteOutConte
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 class LuceneIndexUpdate implements Closeable, LuceneIndexConstants {
 
     private static final Logger log = LoggerFactory
@@ -78,6 +75,7 @@ class LuceneIndexUpdate implements Close
      * text indexing.
      */
     private final Parser parser;
+
     /**
      * The media types supported by the parser used.
      */
@@ -85,68 +83,89 @@ class LuceneIndexUpdate implements Close
 
     private final String path;
 
-    private final NodeBuilder index;
+    private final Set<String> updates = new TreeSet<String>();
 
-    private final Map<String, NodeState> insert = new TreeMap<String, NodeState>();
+    private final IndexWriter writer;
 
-    private final Set<String> remove = new TreeSet<String>();
+    private final Set<Integer> propertyTypes;
 
-    public LuceneIndexUpdate(String path, NodeBuilder index, Parser parser) {
+    public LuceneIndexUpdate(String path, NodeBuilder index, Parser parser)
+            throws CommitFailedException {
         this.path = path;
-        this.index = index;
         this.parser = parser;
+        this.propertyTypes = buildPropertyTypes(index);
+        try {
+            writer = new IndexWriter(new ReadWriteOakDirectory(
+                    index.child(INDEX_DATA_CHILD_NAME)), config);
+        } catch (IOException e) {
+            throw new CommitFailedException("Lucene", 1,
+                    "Failed to update the full text search index", e);
+        }
     }
 
-    public void insert(String path, NodeBuilder value) {
-        Preconditions.checkArgument(path.startsWith(this.path));
-        if (!insert.containsKey(path)) {
-            String key = path.substring(this.path.length());
-            if ("".equals(key)) {
-                key = "/";
-            }
-            // null value can come from a deleted node, followed by a deleted
-            // property event which would trigger an update on the previously
-            // deleted node
-            if (value != null) {
-                insert.put(key, value.getNodeState());
+    private Set<Integer> buildPropertyTypes(NodeBuilder index) {
+        PropertyState ps = index.getProperty(INCLUDE_PROPERTY_TYPES);
+        if (ps == null) {
+            return new HashSet<Integer>();
+        }
+        Set<Integer> includes = new HashSet<Integer>();
+        for (String inc : ps.getValue(Type.STRINGS)) {
+            // TODO add more types as needed
+            if (Type.STRING.toString().equalsIgnoreCase(inc)) {
+                includes.add(Type.STRING.tag());
+            } else if (Type.BINARY.toString().equalsIgnoreCase(inc)) {
+                includes.add(Type.STRING.tag());
             }
         }
+        return includes;
     }
 
-    public void remove(String path) {
-        Preconditions.checkArgument(path.startsWith(this.path));
-        remove.add(path.substring(this.path.length()));
-    }
-
-    public void apply() throws CommitFailedException {
-        if(remove.isEmpty() && insert.isEmpty()){
+    public void insert(String path, NodeBuilder value)
+            throws CommitFailedException {
+        // null value can come from a deleted node, followed by a deleted
+        // property event which would trigger an update on the previously
+        // deleted node
+        if (value == null) {
+            return;
+        }
+        checkArgument(path.startsWith(this.path));
+        String key = path.substring(this.path.length());
+        if ("".equals(key)) {
+            key = "/";
+        }
+        if (!key.startsWith("/")) {
+            key = "/" + key;
+        }
+        if (updates.contains(key)) {
             return;
         }
-        IndexWriter writer = null;
+        updates.add(key);
         try {
-            writer = new IndexWriter(new ReadWriteOakDirectory(
-                    index.child(INDEX_DATA_CHILD_NAME)), config);
-            for (String p : remove) {
-                deleteSubtreeWriter(writer, p);
-            }
-            for (String p : insert.keySet()) {
-                NodeState ns = insert.get(p);
-                addSubtreeWriter(writer, p, ns);
-            }
+            writer.updateDocument(newPathTerm(key),
+                    makeDocument(key, value.getNodeState()));
         } catch (IOException e) {
-            e.printStackTrace();
-            throw new CommitFailedException(
-                    "Lucene", 1,
+            throw new CommitFailedException("Lucene", 1,
                     "Failed to update the full text search index", e);
-        } finally {
-            remove.clear();
-            insert.clear();
-            if (writer != null) {
-                try {
-                    writer.close();
-                } catch (IOException e) {
-                    //
-                }
+        }
+    }
+
+    public void remove(String path) throws CommitFailedException {
+        checkArgument(path.startsWith(this.path));
+        try {
+            deleteSubtreeWriter(writer, path.substring(this.path.length()));
+        } catch (IOException e) {
+            throw new CommitFailedException("Lucene", 1,
+                    "Failed to update the full text search index", e);
+        }
+    }
+
+    public void apply() throws CommitFailedException {
+        if (writer != null) {
+            try {
+                writer.close();
+            } catch (IOException e) {
+                throw new CommitFailedException("Lucene", 1,
+                        "Failed to update the full text search index", e);
             }
         }
     }
@@ -164,28 +183,20 @@ class LuceneIndexUpdate implements Close
         writer.deleteDocuments(new PrefixQuery(newPathTerm(path)));
     }
 
-    private void addSubtreeWriter(IndexWriter writer, String path,
-            NodeState state) throws IOException {
-        if (!path.startsWith("/")) {
-            path = "/" + path;
-        }
-        writer.updateDocument(newPathTerm(path), makeDocument(path, state));
-    }
-
     private Document makeDocument(String path, NodeState state) {
         Document document = new Document();
         document.add(newPathField(path));
         for (PropertyState property : state.getProperties()) {
-            switch (property.getType().tag()) {
-            case PropertyType.BINARY:
-                addBinaryValue(document, property, state);
-                break;
-            default:
-                String pname = property.getName();
-                for (String v : property.getValue(Type.STRINGS)) {
-                    document.add(newPropertyField(pname, v));
+            if (propertyTypes.isEmpty()
+                    || propertyTypes.contains(property.getType().tag())) {
+                if (Type.BINARY.tag() == property.getType().tag()) {
+                    addBinaryValue(document, property, state);
+                } else {
+                    String pname = property.getName();
+                    for (String v : property.getValue(Type.STRINGS)) {
+                        document.add(newPropertyField(pname, v));
+                    }
                 }
-                break;
             }
         }
         return document;
@@ -193,14 +204,14 @@ class LuceneIndexUpdate implements Close
 
     private void addBinaryValue(Document doc, PropertyState property,
             NodeState state) {
-        String type = getOrNull(state, JcrConstants.JCR_MIMETYPE);
+        String type = getString(state, JcrConstants.JCR_MIMETYPE);
         if (type == null || !isSupportedMediaType(type)) {
             return;
         }
         Metadata metadata = new Metadata();
         metadata.set(Metadata.CONTENT_TYPE, type);
         // jcr:encoding is not mandatory
-        String encoding = getOrNull(state, JcrConstants.JCR_ENCODING);
+        String encoding = getString(state, JcrConstants.JCR_ENCODING);
         if (encoding != null) {
             metadata.set(Metadata.CONTENT_ENCODING, encoding);
         }
@@ -211,19 +222,11 @@ class LuceneIndexUpdate implements Close
         }
     }
 
-    private static String getOrNull(NodeState state, String name) {
-        PropertyState p = state.getProperty(name);
-        if (p != null) {
-            return p.getValue(Type.STRING);
-        }
-        return null;
-    }
-
     /**
      * Returns <code>true</code> if the provided type is among the types
      * supported by the Tika parser we are using.
      *
-     * @param type  the type to check.
+     * @param type the type to check.
      * @return whether the type is supported by the Tika parser we are using.
      */
     private boolean isSupportedMediaType(final String type) {
@@ -263,13 +266,12 @@ class LuceneIndexUpdate implements Close
 
     @Override
     public void close() throws IOException {
-        remove.clear();
-        insert.clear();
-    }
-
-    @Override
-    public String toString() {
-        return "LuceneIndexUpdate [path=" + path + ", insert=" + insert
-                + ", remove=" + remove + "]";
+        if (writer != null) {
+            try {
+                writer.close();
+            } catch (IOException e) {
+                //
+            }
+        }
     }
 }

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java?rev=1479514&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java Mon May  6 10:41:23 2013
@@ -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.util;
+
+import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
+import static org.apache.jackrabbit.oak.api.Type.NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NODE_TYPE;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INCLUDE_PROPERTY_TYPES;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.TYPE_LUCENE;
+
+import java.util.Set;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import com.google.common.collect.ImmutableSet;
+
+public class LuceneIndexHelper {
+
+    public static final Set<String> JR_PROPERTY_INCLUDES = ImmutableSet.of(
+            Type.STRING.toString(), Type.BINARY.toString());
+
+    private LuceneIndexHelper() {
+    }
+
+    public static NodeBuilder newLuceneIndexDefinition(
+            @Nonnull NodeBuilder node, @Nonnull String name,
+            @Nullable Set<String> propertyTypes) {
+        NodeBuilder index = node.child(INDEX_DEFINITIONS_NAME);
+        if (index.hasChildNode(name)) {
+            return index.child(name);
+        }
+        index = index.child(name);
+        index.setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE, NAME)
+                .setProperty(TYPE_PROPERTY_NAME, TYPE_LUCENE)
+                .setProperty(REINDEX_PROPERTY_NAME, true);
+        if (propertyTypes != null && !propertyTypes.isEmpty()) {
+            index.setProperty(PropertyStates.createProperty(
+                    INCLUDE_PROPERTY_TYPES, propertyTypes, Type.STRINGS));
+        }
+        return index;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneIndexHelper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneInitializerHelper.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneInitializerHelper.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneInitializerHelper.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/LuceneInitializerHelper.java Mon May  6 10:41:23 2013
@@ -16,42 +16,36 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene.util;
 
-import org.apache.jackrabbit.JcrConstants;
-import org.apache.jackrabbit.oak.api.Type;
-import org.apache.jackrabbit.oak.commons.PathUtils;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper.newLuceneIndexDefinition;
+
+import java.util.Set;
+
 import org.apache.jackrabbit.oak.spi.lifecycle.RepositoryInitializer;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 public class LuceneInitializerHelper implements RepositoryInitializer {
 
-    private final String path;
+    private final String name;
+
+    private final Set<String> propertyTypes;
 
-    public LuceneInitializerHelper(String path) {
-        this.path = path;
+    public LuceneInitializerHelper(String name, Set<String> propertyTypes) {
+        this.name = name;
+        this.propertyTypes = propertyTypes;
     }
 
     @Override
     public NodeState initialize(NodeState state) {
-        NodeBuilder root = state.builder();
-        boolean dirty = false;
-
-        NodeBuilder index = root;
-        for (String p : PathUtils.elements(path)) {
-            if (!index.hasChildNode(p)) {
-                dirty = true;
-            }
-            index = index.child(p);
-        }
-
-        if (dirty) {
-            index.setProperty(JcrConstants.JCR_PRIMARYTYPE,
-                    "oak:queryIndexDefinition", Type.NAME).setProperty("type",
-                    "lucene");
-            index.setProperty("reindex", true);
-            return root.getNodeState();
+        if (state.hasChildNode(INDEX_DEFINITIONS_NAME)
+                && state.getChildNode(INDEX_DEFINITIONS_NAME)
+                        .hasChildNode(name)) {
+            return state;
         }
-        return state;
+        NodeBuilder builder = state.builder();
+        newLuceneIndexDefinition(builder, name, propertyTypes);
+        return builder.getNodeState();
     }
 
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java Mon May  6 10:41:23 2013
@@ -33,7 +33,7 @@ public class LuceneOakRepositoryStub ext
 
     @Override
     protected void preCreateRepository(Jcr jcr) {
-        jcr.with(new LuceneInitializerHelper("/oak:index/luceneGlobal"))
+        jcr.with(new LuceneInitializerHelper("luceneGlobal", null))
                 .with(new LowCostLuceneIndexProvider())
                 .with(new LuceneIndexEditorProvider());
     }

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java Mon May  6 10:41:23 2013
@@ -16,7 +16,6 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
 import org.apache.jackrabbit.oak.spi.query.Filter;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
@@ -28,14 +27,14 @@ import org.apache.jackrabbit.oak.spi.sta
 public class LowCostLuceneIndexProvider extends LuceneIndexProvider {
 
     @Override
-    protected LuceneIndex newLuceneIndex(IndexDefinition child) {
-        return new LowCostLuceneIndex(child);
+    protected LuceneIndex newLuceneIndex() {
+        return new LowCostLuceneIndex();
     }
 
     private static class LowCostLuceneIndex extends LuceneIndex {
 
-        public LowCostLuceneIndex(IndexDefinition indexDefinition) {
-            super(indexDefinition);
+        public LowCostLuceneIndex() {
+            super();
         }
 
         @Override

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java?rev=1479514&r1=1479513&r2=1479514&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java Mon May  6 10:41:23 2013
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertFal
 import static org.junit.Assert.assertTrue;
 
 import java.util.Iterator;
+import java.util.List;
 
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.ContentRepository;
@@ -64,11 +65,6 @@ public class LuceneIndexQueryTest extend
     }
 
     @Test
-    public void xpath() throws Exception {
-        test("xpath.txt");
-    }
-
-    @Test
     public void descendantTest() throws Exception {
         JsopUtil.apply(root, "/ + \"test\": { \"a\": {}, \"b\": {} }");
         root.commit();
@@ -117,4 +113,19 @@ public class LuceneIndexQueryTest extend
         assertEquals("/, /parents", result.next());
         assertFalse(result.hasNext());
     }
+    
+    @Test
+    public void contains() throws Exception {
+        StringBuffer stmt = new StringBuffer();
+        stmt.append("/jcr:root").append("/test").append("/*");
+        stmt.append("[jcr:contains(., '").append("token");
+        stmt.append("')]");
+        System.out.println(stmt.toString());
+
+        List<String> result = executeQuery(stmt.toString(), "xpath");
+        System.out.println(result);
+        for (String h : result) {
+            System.out.println(h);
+        }
+    }
 }

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=1479514&r1=1479513&r2=1479514&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 Mon May  6 10:41:23 2013
@@ -19,14 +19,13 @@ package org.apache.jackrabbit.oak.plugin
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertFalse;
 import static junit.framework.Assert.assertTrue;
-import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
 import static org.apache.jackrabbit.JcrConstants.JCR_SYSTEM;
 import static org.apache.jackrabbit.JcrConstants.NT_BASE;
+import static org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper.newLuceneIndexDefinition;
 import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
 import static org.apache.jackrabbit.oak.plugins.nodetype.NodeTypeConstants.JCR_NODE_TYPES;
 
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinitionImpl;
+import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent;
 import org.apache.jackrabbit.oak.query.ast.Operator;
 import org.apache.jackrabbit.oak.query.ast.SelectorImpl;
@@ -40,7 +39,9 @@ import org.apache.jackrabbit.oak.spi.sta
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.junit.Test;
 
-public class LuceneIndexTest implements LuceneIndexConstants {
+import com.google.common.collect.ImmutableSet;
+
+public class LuceneIndexTest {
 
     private NodeState root = new InitialContent().initialize(EMPTY_NODE);
 
@@ -48,20 +49,16 @@ public class LuceneIndexTest implements 
 
     @Test
     public void testLucene() throws Exception {
-        builder.child("oak:index").child("lucene")
-                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE)
-                .setProperty("type", TYPE_LUCENE);
+        newLuceneIndexDefinition(builder, "lucene", null);
 
         NodeState before = builder.getNodeState();
         builder.setProperty("foo", "bar");
         NodeState after = builder.getNodeState();
 
-        EditorDiff.process(new LuceneIndexDiff(builder), before, after);
+        EditorDiff.process(new LuceneIndexEditor(builder), before, after);
         NodeState indexed = builder.getNodeState();
 
-        IndexDefinition testDef = new IndexDefinitionImpl("lucene",
-                TYPE_LUCENE, "/oak:index/lucene");
-        QueryIndex queryIndex = new LuceneIndex(testDef);
+        QueryIndex queryIndex = new LuceneIndex();
         FilterImpl filter = createFilter(NT_BASE);
         filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,
@@ -74,9 +71,7 @@ public class LuceneIndexTest implements 
 
     @Test
     public void testLucene2() throws Exception {
-        builder.child("oak:index").child("lucene")
-                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE)
-                .setProperty("type", TYPE_LUCENE);
+        newLuceneIndexDefinition(builder, "lucene", null);
 
         NodeState before = builder.getNodeState();
         builder.setProperty("foo", "bar");
@@ -86,12 +81,10 @@ public class LuceneIndexTest implements 
 
         NodeState after = builder.getNodeState();
 
-        EditorDiff.process(new LuceneIndexDiff(builder), before, after);
+        EditorDiff.process(new LuceneIndexEditor(builder), before, after);
         NodeState indexed = builder.getNodeState();
 
-        IndexDefinition testDef = new IndexDefinitionImpl("lucene",
-                TYPE_LUCENE, "/oak:index/lucene");
-        QueryIndex queryIndex = new LuceneIndex(testDef);
+        QueryIndex queryIndex = new LuceneIndex();
         FilterImpl filter = createFilter(NT_BASE);
         // filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,
@@ -106,6 +99,34 @@ public class LuceneIndexTest implements 
         assertFalse(cursor.hasNext());
     }
 
+    @Test
+    public void testLucene3() throws Exception {
+        newLuceneIndexDefinition(builder, "lucene", ImmutableSet.of(Type.STRING.toString()));
+
+        NodeState before = builder.getNodeState();
+        builder.setProperty("foo", "bar");
+        builder.child("a").setProperty("foo", "bar");
+        builder.child("a").child("b").setProperty("foo", "bar", Type.NAME);
+        builder.child("a").child("b").child("c").setProperty("foo", "bar", Type.NAME);
+
+        NodeState after = builder.getNodeState();
+
+        EditorDiff.process(new LuceneIndexEditor(builder), before, after);
+        NodeState indexed = builder.getNodeState();
+
+        QueryIndex queryIndex = new LuceneIndex();
+        FilterImpl filter = createFilter(NT_BASE);
+        // filter.restrictPath("/", Filter.PathRestriction.EXACT);
+        filter.restrictProperty("foo", Operator.EQUAL,
+                PropertyValues.newString("bar"));
+        Cursor cursor = queryIndex.query(filter, indexed);
+
+        assertTrue(cursor.hasNext());
+        assertEquals("/", cursor.next().getPath());
+        assertEquals("/a", cursor.next().getPath());
+        assertFalse(cursor.hasNext());
+    }
+
     private FilterImpl createFilter(String nodeTypeName) {
         NodeState system = root.getChildNode(JCR_SYSTEM);
         NodeState types = system.getChildNode(JCR_NODE_TYPES);