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 2012/10/25 11:13:02 UTC

svn commit: r1402036 - in /jackrabbit/oak/trunk: oak-core/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/i...

Author: alexparvulescu
Date: Thu Oct 25 09:13:00 2012
New Revision: 1402036

URL: http://svn.apache.org/viewvc?rev=1402036&view=rev
Log:
OAK-394 IndexManagerHook to manage existing indexes

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java   (with props)
Removed:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneReindexHook.java
Modified:
    jackrabbit/oak/trunk/oak-core/pom.xml
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexConstants.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneEditor.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneHook.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexDiff.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHook.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/nodetype/InitialContent.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/old/QueryTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexQueryTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/query/AbstractQueryTest.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/security/user/UserProviderTest.java
    jackrabbit/oak/trunk/oak-jcr/src/main/java/org/apache/jackrabbit/oak/jcr/Jcr.java
    jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/run/Main.java

Modified: jackrabbit/oak/trunk/oak-core/pom.xml
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/pom.xml?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/pom.xml (original)
+++ jackrabbit/oak/trunk/oak-core/pom.xml Thu Oct 25 09:13:00 2012
@@ -48,6 +48,7 @@
                 org.apache.jackrabbit.oak.plugins.value,
                 org.apache.jackrabbit.oak.plugins.commit,
                 org.apache.jackrabbit.oak.plugins.identifier,
+                org.apache.jackrabbit.oak.plugins.index,
                 org.apache.jackrabbit.oak.plugins.index.lucene,
                 org.apache.jackrabbit.oak.plugins.index.property,
                 org.apache.jackrabbit.oak.plugins.memory,

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class CompositeIndexHookProvider implements IndexHookProvider {
+
+    @Nonnull
+    public static IndexHookProvider compose(
+            @Nonnull Collection<IndexHookProvider> providers) {
+        if (providers.isEmpty()) {
+            return new IndexHookProvider() {
+                @Override
+                public List<? extends IndexHook> getIndexHooks(String type,
+                        NodeBuilder builder) {
+                    return ImmutableList.of();
+                }
+            };
+        } else if (providers.size() == 1) {
+            return providers.iterator().next();
+        } else {
+            return new CompositeIndexHookProvider(
+                    ImmutableList.copyOf(providers));
+        }
+    }
+
+    private final List<IndexHookProvider> providers;
+
+    private CompositeIndexHookProvider(List<IndexHookProvider> providers) {
+        this.providers = providers;
+    }
+
+    public CompositeIndexHookProvider(IndexHookProvider... providers) {
+        this(Arrays.asList(providers));
+    }
+
+    @Override
+    @Nonnull
+    public List<? extends IndexHook> getIndexHooks(String type,
+            NodeBuilder builder) {
+        List<IndexHook> indexes = Lists.newArrayList();
+        for (IndexHookProvider provider : providers) {
+            indexes.addAll(provider.getIndexHooks(type, builder));
+        }
+        return indexes;
+    }
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/CompositeIndexHookProvider.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexConstants.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexConstants.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexConstants.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexConstants.java Thu Oct 25 09:13:00 2012
@@ -28,9 +28,4 @@ public interface IndexConstants {
 
     String REINDEX_PROPERTY_NAME = "reindex";
 
-    String INDEX_DATA_CHILD_NAME = ":data";
-
-    //TODO remove this property as soon as the index manager is in
-    String DEFAULT_INDEX_HOME = "/";
-
 }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,23 @@
+/*
+ * 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;
+
+import org.apache.jackrabbit.oak.spi.commit.CommitHook;
+
+public interface IndexHook extends CommitHook {
+
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,223 @@
+/*
+ * 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;
+
+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.IndexConstants.TYPE_UNKNOWN;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_PROPERTY_NAME;
+
+import java.util.HashMap;
+import java.util.List;
+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.memory.MemoryNodeState;
+import org.apache.jackrabbit.oak.spi.commit.CommitHook;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStateDiff;
+import org.apache.jackrabbit.oak.spi.state.NodeStateUtils;
+
+public class IndexHookManager implements CommitHook {
+
+    private final IndexHookProvider provider;
+
+    public IndexHookManager(IndexHookProvider provider) {
+        this.provider = provider;
+    }
+
+    @Override
+    public NodeState processCommit(NodeState before, NodeState after)
+            throws CommitFailedException {
+        NodeBuilder builder = after.builder();
+
+        // <path>, <builder>
+        Map<String, NodeBuilder> changedDefs = new HashMap<String, NodeBuilder>();
+        // <path>, <builder>
+        Map<String, NodeBuilder> existingDefs = new HashMap<String, NodeBuilder>();
+
+        IndexDefDiff diff = new IndexDefDiff(builder, changedDefs, existingDefs);
+        after.compareAgainstBaseState(before, diff);
+
+        // <path>, <builder>
+        Map<String, NodeBuilder> allDefs = new HashMap<String, NodeBuilder>(
+                changedDefs);
+        allDefs.putAll(existingDefs);
+
+        // <type, <<path>, <builder>>
+        Map<String, Map<String, NodeBuilder>> updates = new HashMap<String, Map<String, NodeBuilder>>();
+        for (String def : allDefs.keySet()) {
+            NodeBuilder cb = allDefs.get(def);
+            String type = TYPE_UNKNOWN;
+            PropertyState typePS = cb.getProperty(TYPE_PROPERTY_NAME);
+            if (typePS != null && !typePS.isArray()) {
+                type = typePS.getValue(Type.STRING);
+            }
+            Map<String, NodeBuilder> defs = updates.get(type);
+            if (defs == null) {
+                defs = new HashMap<String, NodeBuilder>();
+                updates.put(type, defs);
+            }
+            defs.put(def, cb);
+        }
+
+        // commit
+        for (String type : updates.keySet()) {
+            Map<String, NodeBuilder> indexDefs = updates.get(type);
+            for (String p : indexDefs.keySet()) {
+                List<? extends IndexHook> hooks = provider.getIndexHooks(type,
+                        indexDefs.get(p));
+                for (IndexHook hook : hooks) {
+                    boolean reindex = changedDefs.keySet().contains(p);
+                    if (reindex) {
+                        hook.processCommit(MemoryNodeState.EMPTY_NODE, after);
+                    } else {
+                        hook.processCommit(before, after);
+                    }
+                }
+            }
+        }
+        return builder.getNodeState();
+    }
+
+    protected static class IndexDefDiff implements NodeStateDiff {
+
+        private final IndexDefDiff parent;
+
+        private final NodeBuilder node;
+
+        private final String name;
+
+        private String path;
+
+        private final Map<String, NodeBuilder> updates;
+        private final Map<String, NodeBuilder> existing;
+
+        public IndexDefDiff(IndexDefDiff parent, NodeBuilder node, String name,
+                String path, Map<String, NodeBuilder> updates,
+                Map<String, NodeBuilder> existing) {
+            this.parent = parent;
+            this.node = node;
+            this.name = name;
+            this.path = path;
+            this.updates = updates;
+            this.existing = existing;
+
+            if (node != null
+                    && isIndexNodeType(node.getProperty(JCR_PRIMARYTYPE))) {
+                getAndResetReindex(node);
+                this.updates.put(getPath(), node);
+                this.existing.remove(getPath());
+            }
+
+            if (node != null && node.hasChildNode(INDEX_DEFINITIONS_NAME)) {
+                NodeBuilder index = node.child(INDEX_DEFINITIONS_NAME);
+                for (String indexName : index.getChildNodeNames()) {
+                    String indexPath = concat(getPath(),
+                            INDEX_DEFINITIONS_NAME, indexName);
+                    NodeBuilder indexChild = index.child(indexName);
+                    if (isIndexNodeType(indexChild.getProperty(JCR_PRIMARYTYPE))) {
+                        boolean reindex = getAndResetReindex(indexChild);
+                        if (reindex) {
+                            this.updates.put(indexPath, indexChild);
+                        } else {
+                            this.existing.put(indexPath, indexChild);
+                        }
+                    }
+                }
+            }
+        }
+
+        public IndexDefDiff(NodeBuilder root, Map<String, NodeBuilder> updates,
+                Map<String, NodeBuilder> existing) {
+            this(null, root, null, "/", updates, existing);
+        }
+
+        public IndexDefDiff(IndexDefDiff parent, String name) {
+            this(parent, getChildNode(parent.node, name), name, null,
+                    parent.updates, parent.existing);
+        }
+
+        private static NodeBuilder getChildNode(NodeBuilder node, String name) {
+            if (node != null && node.hasChildNode(name)) {
+                return node.child(name);
+            } else {
+                return null;
+            }
+        }
+
+        private String getPath() {
+            if (path == null) { // => parent != null
+                path = concat(parent.getPath(), name);
+            }
+            return path;
+        }
+
+        @Override
+        public void propertyAdded(PropertyState after) {
+        }
+
+        @Override
+        public void propertyChanged(PropertyState before, PropertyState after) {
+        }
+
+        @Override
+        public void propertyDeleted(PropertyState before) {
+        }
+
+        @Override
+        public void childNodeAdded(String name, NodeState after) {
+            childNodeChanged(name, MemoryNodeState.EMPTY_NODE, after);
+        }
+
+        @Override
+        public void childNodeChanged(String name, NodeState before,
+                NodeState after) {
+            if (NodeStateUtils.isHidden(name)) {
+                return;
+            }
+            after.compareAgainstBaseState(before, new IndexDefDiff(this, name));
+        }
+
+        @Override
+        public void childNodeDeleted(String name, NodeState before) {
+        }
+
+        private boolean isIndexNodeType(PropertyState ps) {
+            return ps != null
+                    && !ps.isArray()
+                    && ps.getValue(Type.STRING).equals(
+                            INDEX_DEFINITIONS_NODE_TYPE);
+        }
+
+        private boolean getAndResetReindex(NodeBuilder builder) {
+            boolean isReindex = false;
+            PropertyState ps = builder.getProperty(REINDEX_PROPERTY_NAME);
+            if (ps != null) {
+                isReindex = ps.getValue(Type.BOOLEAN);
+                builder.setProperty(REINDEX_PROPERTY_NAME, false);
+            }
+            return isReindex;
+        }
+    }
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+public interface IndexHookProvider {
+
+    @Nonnull
+    List<? extends IndexHook> getIndexHooks(String type, NodeBuilder builder);
+
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookProvider.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneEditor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneEditor.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneEditor.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneEditor.java Thu Oct 25 09:13:00 2012
@@ -17,7 +17,6 @@
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
-import static org.apache.jackrabbit.oak.commons.PathUtils.elements;
 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;
@@ -71,23 +70,17 @@ class LuceneEditor implements CommitHook
         }
     }
 
-    private final String path;
+    private final NodeBuilder root;
 
-    public LuceneEditor(String path) {
-        this.path = path;
+    public LuceneEditor(NodeBuilder root) {
+        this.root = root;
     }
 
     @Override
     public NodeState processCommit(NodeState before, NodeState after)
             throws CommitFailedException {
-        NodeBuilder rootBuilder = after.builder();
-        NodeBuilder builder = rootBuilder;
-        for (String name : elements(path)) {
-            builder = builder.child(name);
-        }
-        builder = builder.child(INDEX_DATA_CHILD_NAME);
-        Directory directory = new ReadWriteOakDirectory(builder);
-
+        Directory directory = new ReadWriteOakDirectory(
+                root.child(INDEX_DATA_CHILD_NAME));
         try {
             IndexWriter writer = new IndexWriter(directory, config);
             try {
@@ -96,9 +89,8 @@ class LuceneEditor implements CommitHook
                 diff.postProcess(after);
             } finally {
                 writer.close();
-                builder.setProperty(REINDEX_PROPERTY_NAME, false);
             }
-            return rootBuilder.getNodeState();
+            return after;
         } catch (IOException e) {
             e.printStackTrace();
             throw new CommitFailedException(

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneHook.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneHook.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneHook.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneHook.java Thu Oct 25 09:13:00 2012
@@ -16,35 +16,31 @@
  */
 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 org.apache.jackrabbit.oak.api.CommitFailedException;
-import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
-import org.apache.jackrabbit.oak.spi.commit.CommitHook;
-import org.apache.jackrabbit.oak.spi.commit.CompositeHook;
+import org.apache.jackrabbit.oak.plugins.index.IndexHook;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
-public class LuceneHook implements CommitHook, LuceneIndexConstants {
+/**
+ * {@link IndexHook} implementation that is responsible for keeping the
+ * {@link LuceneIndex} up to date
+ * 
+ * @see LuceneIndex
+ * 
+ */
+public class LuceneHook implements IndexHook {
 
-    private final String indexConfigPath;
+    private final NodeBuilder builder;
 
-    public LuceneHook(String indexConfigPath) {
-        this.indexConfigPath = indexConfigPath;
+    public LuceneHook(NodeBuilder builder) {
+        this.builder = builder;
     }
 
     @Override
     public NodeState processCommit(NodeState before, NodeState after)
             throws CommitFailedException {
-        List<CommitHook> hooks = new ArrayList<CommitHook>();
-        List<IndexDefinition> indexDefinitions = buildIndexDefinitions(after,
-                indexConfigPath, TYPE_LUCENE);
-        for (IndexDefinition def : indexDefinitions) {
-            hooks.add(new LuceneEditor(def.getPath()));
-        }
-        return CompositeHook.compose(hooks).processCommit(before, after);
+        new LuceneEditor(builder).processCommit(before, after);
+        return after;
     }
 
 }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java Thu Oct 25 09:13:00 2012
@@ -22,6 +22,6 @@ public interface LuceneIndexConstants ex
 
     String TYPE_LUCENE = "lucene";
 
-    String DEFAULT_INDEX_NAME = "default-lucene";
+    String INDEX_DATA_CHILD_NAME = ":data";
 
 }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import org.apache.jackrabbit.oak.plugins.index.IndexHook;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookProvider;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import com.google.common.collect.ImmutableList;
+
+public class LuceneIndexHookProvider implements IndexHookProvider,
+        LuceneIndexConstants {
+
+    @Override
+    public List<? extends IndexHook> getIndexHooks(String type,
+            NodeBuilder builder) {
+        if (TYPE_LUCENE.equals(type)) {
+            return ImmutableList.of(new LuceneHook(builder));
+        }
+        return ImmutableList.of();
+    }
+
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexHookProvider.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java Thu Oct 25 09:13:00 2012
@@ -19,12 +19,10 @@ package org.apache.jackrabbit.oak.plugin
 import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.buildIndexDefinitions;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import javax.annotation.Nonnull;
 
-import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
@@ -41,21 +39,11 @@ public class LuceneIndexProvider impleme
     private static final Logger LOG = LoggerFactory
             .getLogger(LuceneIndexProvider.class);
 
-    private final String indexPath;
-
-    public LuceneIndexProvider(String indexPath) {
-        this.indexPath = indexPath;
-    }
-
     @Override @Nonnull
     public List<QueryIndex> getQueryIndexes(NodeState nodeState) {
-        if (!PathUtils.isValid(indexPath)) {
-            LOG.warn("index path is not valid {}", indexPath);
-            return Collections.<QueryIndex> emptyList();
-        }
         List<QueryIndex> tempIndexes = new ArrayList<QueryIndex>();
-        for (IndexDefinition child : buildIndexDefinitions(nodeState,
-                indexPath, TYPE_LUCENE)) {
+        for (IndexDefinition child : buildIndexDefinitions(nodeState, "/",
+                TYPE_LUCENE)) {
             LOG.debug("found a lucene index definition {}", child);
             tempIndexes.add(new LuceneIndex(child));
         }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexDiff.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexDiff.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexDiff.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexDiff.java Thu Oct 25 09:13:00 2012
@@ -16,7 +16,9 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.property;
 
+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.INDEX_DEFINITIONS_NODE_TYPE;
 import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
 
 import java.util.List;
@@ -61,21 +63,13 @@ class PropertyIndexDiff implements NodeS
         this.path = path;
         this.updates = updates;
 
+        if (node != null && isIndexNodeType(node.getProperty(JCR_PRIMARYTYPE))) {
+            update(node, name);
+        }
         if (node != null && node.hasChildNode(INDEX_DEFINITIONS_NAME)) {
             NodeBuilder index = node.child(INDEX_DEFINITIONS_NAME);
             for (String indexName : index.getChildNodeNames()) {
-                NodeBuilder indexChild = index.child(indexName);
-                PropertyState ps = indexChild.getProperty("propertyNames");
-                Iterable<String> propertyNames = ps != null ? ps
-                        .getValue(Type.STRINGS) : ImmutableList.of(indexName);
-                for (String pname : propertyNames) {
-                    List<PropertyIndexUpdate> list = this.updates.get(pname);
-                    if (list == null) {
-                        list = Lists.newArrayList();
-                        this.updates.put(pname, list);
-                    }
-                    list.add(new PropertyIndexUpdate(getPath(), indexChild));
-                }
+                update(index.child(indexName), indexName);
             }
         }
     }
@@ -114,6 +108,25 @@ class PropertyIndexDiff implements NodeS
         }
     }
 
+    private void update(NodeBuilder builder, String indexName) {
+        PropertyState ps = builder.getProperty("propertyNames");
+        Iterable<String> propertyNames = ps != null ? ps.getValue(Type.STRINGS)
+                : ImmutableList.of(indexName);
+        for (String pname : propertyNames) {
+            List<PropertyIndexUpdate> list = this.updates.get(pname);
+            if (list == null) {
+                list = Lists.newArrayList();
+                this.updates.put(pname, list);
+            }
+            list.add(new PropertyIndexUpdate(getPath(), builder));
+        }
+    }
+
+    private boolean isIndexNodeType(PropertyState ps) {
+        return ps != null && !ps.isArray()
+                && ps.getValue(Type.STRING).equals(INDEX_DEFINITIONS_NODE_TYPE);
+    }
+
     //-----------------------------------------------------< NodeStateDiff >--
 
     @Override

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHook.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHook.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHook.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHook.java Thu Oct 25 09:13:00 2012
@@ -22,27 +22,31 @@ import java.util.Map;
 import javax.annotation.Nonnull;
 
 import org.apache.jackrabbit.oak.api.CommitFailedException;
-import org.apache.jackrabbit.oak.spi.commit.CommitHook;
+import org.apache.jackrabbit.oak.plugins.index.IndexHook;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 import com.google.common.collect.Maps;
 
 /**
- * {@link CommitHook} implementation that is responsible for keeping the
+ * {@link IndexHook} implementation that is responsible for keeping the
  * {@link PropertyIndex} up to date
  * 
  * @see PropertyIndex
  * @see PropertyIndexLookup
  * 
  */
-public class PropertyIndexHook implements CommitHook {
+public class PropertyIndexHook implements IndexHook {
+    
+    private final NodeBuilder builder;
+
+    public PropertyIndexHook(NodeBuilder builder) {
+        this.builder = builder;
+    }
 
     @Override @Nonnull
     public NodeState processCommit(NodeState before, NodeState after)
             throws CommitFailedException {
-        NodeBuilder builder = after.builder();
-
         Map<String, List<PropertyIndexUpdate>> indexes = Maps.newHashMap();
         PropertyIndexDiff diff = new PropertyIndexDiff(builder, indexes);
         after.compareAgainstBaseState(before, diff);
@@ -53,7 +57,7 @@ public class PropertyIndexHook implement
             }
         }
 
-        return builder.getNodeState();
+        return after;
     }
 
 

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.jackrabbit.oak.plugins.index.property;
+
+import java.util.List;
+
+import org.apache.jackrabbit.oak.plugins.index.IndexHook;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookProvider;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import com.google.common.collect.ImmutableList;
+
+public class PropertyIndexHookProvider implements IndexHookProvider {
+
+    private static final String TYPE = "property";
+
+    @Override
+    public List<? extends IndexHook> getIndexHooks(String type,
+            NodeBuilder builder) {
+        if (TYPE.equals(type)) {
+            return ImmutableList.of(new PropertyIndexHook(builder));
+        }
+        return ImmutableList.of();
+    }
+
+}
\ No newline at end of file

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

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexHookProvider.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/nodetype/InitialContent.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/nodetype/InitialContent.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/nodetype/InitialContent.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/nodetype/InitialContent.java Thu Oct 25 09:13:00 2012
@@ -91,6 +91,7 @@ public class InitialContent implements R
             NodeBuilder index = root.child("oak:index");
             index.child("uuid")
                 .setProperty("jcr:primaryType", "oak:queryIndexDefinition", Type.NAME)
+                .setProperty("type", "property")
                 .setProperty("propertyNames", "jcr:uuid")
                 .setProperty("unique", true);
             index.child("primaryType")
@@ -100,10 +101,12 @@ public class InitialContent implements R
             // FIXME: rep:principalName only needs to be unique if defined with user/group nodes -> add defining nt-info to uniqueness constraint otherwise ac-editing will fail.
             index.child("authorizableId")
                 .setProperty("jcr:primaryType", "oak:queryIndexDefinition", Type.NAME)
+                .setProperty("type", "property")
                 .setProperty("propertyNames", "rep:authorizableId")
                 .setProperty("unique", true);
             index.child("principalName")
                 .setProperty("jcr:primaryType", "oak:queryIndexDefinition", Type.NAME)
+                .setProperty("type", "property")
                 .setProperty("propertyNames", "rep:principalName")
                 .setProperty("unique", true);
         }

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java?rev=1402036&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java Thu Oct 25 09:13:00 2012
@@ -0,0 +1,123 @@
+/*
+ * 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;
+
+import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NODE_TYPE;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager.IndexDefDiff;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.junit.Test;
+
+public class IndexHookManagerTest {
+
+    @Test
+    public void test() throws Exception {
+        NodeState root = MemoryNodeState.EMPTY_NODE;
+
+        NodeBuilder builder = root.builder();
+        // this index is on the current update branch, it should be seen by the
+        // diff
+        builder.child("oak:index")
+                .child("existing")
+                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE,
+                        Type.NAME);
+        // this index is NOT the current update branch, it should NOT be seen by
+        // the diff
+        builder.child("newchild")
+                .child("other")
+                .child("oak:index")
+                .child("existing2")
+                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE,
+                        Type.NAME);
+
+        NodeState before = builder.getNodeState();
+        // Add index definition
+        builder.child("oak:index")
+                .child("foo")
+                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE,
+                        Type.NAME);
+        builder.child("test")
+                .child("other")
+                .child("oak:index")
+                .child("index2")
+                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE,
+                        Type.NAME);
+        NodeState after = builder.getNodeState();
+
+        // <path>, <state>
+        Map<String, NodeBuilder> changedDefs = new HashMap<String, NodeBuilder>();
+        // <path>, <state>
+        Map<String, NodeBuilder> existingDefs = new HashMap<String, NodeBuilder>();
+
+        IndexDefDiff diff = new IndexDefDiff(builder, changedDefs, existingDefs);
+        after.compareAgainstBaseState(before, diff);
+
+        Set<String> updates = changedDefs.keySet();
+        String[] expected = new String[] { "/oak:index/foo",
+                "/test/other/oak:index/index2" };
+        for (String def : expected) {
+            assertTrue("Expected to find " + def, updates.remove(def));
+        }
+        assertTrue(updates.isEmpty());
+
+        Set<String> existing = existingDefs.keySet();
+        String[] expectedE = new String[] { "/oak:index/existing", };
+        for (String def : expectedE) {
+            assertTrue("Expected to find " + def, existing.remove(def));
+        }
+        assertTrue(existing.isEmpty());
+    }
+
+    @Test
+    public void testReindex() throws Exception {
+        NodeState root = MemoryNodeState.EMPTY_NODE;
+
+        NodeBuilder builder = root.builder();
+        builder.child("oak:index")
+                .child("reindexed")
+                .setProperty(JCR_PRIMARYTYPE, INDEX_DEFINITIONS_NODE_TYPE,
+                        Type.NAME)
+                .setProperty(IndexConstants.REINDEX_PROPERTY_NAME, true);
+
+        NodeState state = builder.getNodeState();
+
+        // <path>, <state>
+        Map<String, NodeBuilder> changedDefs = new HashMap<String, NodeBuilder>();
+        // <path>, <state>
+        Map<String, NodeBuilder> existingDefs = new HashMap<String, NodeBuilder>();
+
+        IndexDefDiff diff = new IndexDefDiff(builder, changedDefs, existingDefs);
+        state.compareAgainstBaseState(state, diff);
+
+        Set<String> updates = changedDefs.keySet();
+        String[] expected = new String[] { "/oak:index/reindexed" };
+        for (String def : expected) {
+            assertTrue("Expected to find " + def, updates.remove(def));
+        }
+        assertTrue(updates.isEmpty());
+        assertTrue(existingDefs.isEmpty());
+    }
+}

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

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerTest.java
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Revision Rev URL

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java Thu Oct 25 09:13:00 2012
@@ -19,6 +19,7 @@ package org.apache.jackrabbit.oak.plugin
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.ContentRepository;
 import org.apache.jackrabbit.oak.api.Tree;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager;
 import org.apache.jackrabbit.oak.plugins.nodetype.InitialContent;
 import org.apache.jackrabbit.oak.query.AbstractQueryTest;
 
@@ -39,9 +40,8 @@ public class LuceneIndexQueryTest extend
     protected ContentRepository createRepository() {
         return new Oak()
             .with(new InitialContent())
-            .with(new LuceneIndexProvider(TEST_INDEX_HOME))
-            .with(new LuceneReindexHook(TEST_INDEX_HOME))
-            .with(new LuceneHook(TEST_INDEX_HOME))
+            .with(new LuceneIndexProvider())
+            .with(new IndexHookManager(new LuceneIndexHookProvider()))
             .createContentRepository();
     }
 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java Thu Oct 25 09:13:00 2012
@@ -20,52 +20,42 @@ import static junit.framework.Assert.ass
 import static junit.framework.Assert.assertFalse;
 import static junit.framework.Assert.assertTrue;
 
-import javax.security.auth.Subject;
-
-import org.apache.jackrabbit.mk.core.MicroKernelImpl;
-import org.apache.jackrabbit.oak.api.Root;
-import org.apache.jackrabbit.oak.api.Tree;
-import org.apache.jackrabbit.oak.core.RootImpl;
-import org.apache.jackrabbit.oak.kernel.KernelNodeStore;
 import org.apache.jackrabbit.oak.plugins.index.IndexDefinition;
 import org.apache.jackrabbit.oak.plugins.index.IndexDefinitionImpl;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeState;
 import org.apache.jackrabbit.oak.query.ast.Operator;
 import org.apache.jackrabbit.oak.query.index.FilterImpl;
-import org.apache.jackrabbit.oak.security.authorization.AccessControlProviderImpl;
-import org.apache.jackrabbit.oak.spi.query.CompositeQueryIndexProvider;
 import org.apache.jackrabbit.oak.spi.query.Cursor;
 import org.apache.jackrabbit.oak.spi.query.Filter;
 import org.apache.jackrabbit.oak.spi.query.PropertyValues;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.junit.Test;
 
 public class LuceneIndexTest implements LuceneIndexConstants {
 
-    private static String DEFAULT_INDEX_NAME = "default-lucene";
-
     @Test
     public void testLucene() throws Exception {
-        KernelNodeStore store = new KernelNodeStore(new MicroKernelImpl());
+        NodeState root = MemoryNodeState.EMPTY_NODE;
+
+        NodeBuilder builder = root.builder();
+        NodeBuilder index = builder.child("oak:index").child("lucene");
+        IndexDefinition testDef = new IndexDefinitionImpl("lucene",
+                TYPE_LUCENE, "/oak:index/lucene");
+
+        NodeState before = builder.getNodeState();
+        builder.setProperty("foo", "bar");
+        NodeState after = builder.getNodeState();
 
-        IndexDefinition testID = new IndexDefinitionImpl(DEFAULT_INDEX_NAME,
-                TYPE_LUCENE, "/" + INDEX_DEFINITIONS_NAME + "/"
-                        + DEFAULT_INDEX_NAME);
-
-        store.setHook(new LuceneEditor(testID.getPath()));
-        Root root = new RootImpl(store, null, new Subject(),
-                new AccessControlProviderImpl(),
-                new CompositeQueryIndexProvider());
-
-        Tree tree = root.getTree("/");
-        tree.setProperty("foo", "bar");
-        root.commit();
+        new LuceneEditor(index).processCommit(before, after);
 
-        QueryIndex index = new LuceneIndex(testID);
+        QueryIndex queryIndex = new LuceneIndex(testDef);
         FilterImpl filter = new FilterImpl(null);
         filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,
                 PropertyValues.newString("bar"));
-        Cursor cursor = index.query(filter, store.getRoot());
+        Cursor cursor = queryIndex.query(filter, builder.getNodeState());
         assertTrue(cursor.next());
         assertEquals("/", cursor.currentRow().getPath());
         assertFalse(cursor.next());

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/old/QueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/old/QueryTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/old/QueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/old/QueryTest.java Thu Oct 25 09:13:00 2012
@@ -19,8 +19,6 @@ import org.apache.jackrabbit.mk.core.Mic
 import org.apache.jackrabbit.mk.index.IndexWrapper;
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.ContentRepository;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneHook;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneReindexHook;
 import org.apache.jackrabbit.oak.plugins.nodetype.InitialContent;
 import org.apache.jackrabbit.oak.query.AbstractQueryTest;
 import org.apache.jackrabbit.oak.spi.commit.CommitHook;
@@ -37,7 +35,7 @@ public class QueryTest extends AbstractQ
     protected ContentRepository createRepository() {
         // the property and prefix index currently require the index wrapper
         IndexWrapper mk = new IndexWrapper(new MicroKernelImpl(),
-                TEST_INDEX_HOME + INDEX_DEFINITIONS_NAME + "/indexes");
+                "/" + INDEX_DEFINITIONS_NAME + "/indexes");
 
         PropertyIndexer indexer = new PropertyIndexer(mk.getIndexer());
 
@@ -45,8 +43,6 @@ public class QueryTest extends AbstractQ
             .with(new InitialContent())
             .with((QueryIndexProvider) indexer)
             .with((CommitHook) indexer)
-            .with(new LuceneReindexHook(TEST_INDEX_HOME))
-            .with(new LuceneHook(TEST_INDEX_HOME))
             .createContentRepository();
     }
 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexQueryTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexQueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexQueryTest.java Thu Oct 25 09:13:00 2012
@@ -18,6 +18,7 @@ package org.apache.jackrabbit.oak.plugin
 
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.ContentRepository;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager;
 import org.apache.jackrabbit.oak.plugins.nodetype.InitialContent;
 import org.apache.jackrabbit.oak.query.AbstractQueryTest;
 
@@ -32,7 +33,7 @@ public class PropertyIndexQueryTest exte
         return new Oak()
             .with(new InitialContent())
             .with(new PropertyIndexProvider())
-            .with(new PropertyIndexHook())
+            .with(new IndexHookManager(new PropertyIndexHookProvider()))
             .createContentRepository();
     }
 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/property/PropertyIndexTest.java Thu Oct 25 09:13:00 2012
@@ -61,7 +61,7 @@ public class PropertyIndexTest {
 
         // ... then see how adding an index affects the code
         lookup = new PropertyIndexLookup(
-                new PropertyIndexHook().processCommit(before, after));
+                new PropertyIndexHook(builder).processCommit(before, after));
         long withIndex = System.nanoTime();
         assertEquals(ImmutableSet.of("a", "b"), lookup.find("foo", "abc"));
         assertEquals(ImmutableSet.of("b"), lookup.find("foo", "def"));
@@ -105,7 +105,7 @@ public class PropertyIndexTest {
 
         // ... then see how adding an index affects the code
         lookup = new PropertyIndexLookup(
-                new PropertyIndexHook().processCommit(before, after));
+                new PropertyIndexHook(builder).processCommit(before, after));
         long withIndex = System.nanoTime();
         assertEquals(ImmutableSet.of("a", "b"), lookup.find("foo", "abc"));
         assertEquals(ImmutableSet.of("b"), lookup.find("foo", "def"));

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/query/AbstractQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/query/AbstractQueryTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/query/AbstractQueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/query/AbstractQueryTest.java Thu Oct 25 09:13:00 2012
@@ -41,7 +41,6 @@ import org.apache.jackrabbit.oak.api.Roo
 import org.apache.jackrabbit.oak.api.SessionQueryEngine;
 import org.apache.jackrabbit.oak.api.Tree;
 import org.apache.jackrabbit.oak.api.Type;
-import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.spi.query.PropertyValues;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -60,7 +59,6 @@ import static org.junit.Assert.fail;
  */
 public abstract class AbstractQueryTest {
 
-    public static final String TEST_INDEX_HOME = "/";
     protected static final String TEST_INDEX_NAME = "test-index";
 
     protected SessionQueryEngine qe;
@@ -90,15 +88,7 @@ public abstract class AbstractQueryTest 
 
     protected static Tree createTestIndexNode(Tree index, String type)
             throws Exception {
-        Tree indexDef = index;
-        for (String p : PathUtils.elements(TEST_INDEX_HOME)) {
-            if (indexDef.hasChild(p)) {
-                indexDef = indexDef.getChild(p);
-            } else {
-                indexDef = indexDef.addChild(p);
-            }
-        }
-        indexDef = indexDef.addChild(INDEX_DEFINITIONS_NAME).addChild(
+        Tree indexDef = index.addChild(INDEX_DEFINITIONS_NAME).addChild(
                 TEST_INDEX_NAME);
         indexDef.setProperty(JcrConstants.JCR_PRIMARYTYPE,
                 INDEX_DEFINITIONS_NODE_TYPE, Type.NAME);

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/security/user/UserProviderTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/security/user/UserProviderTest.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/security/user/UserProviderTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/security/user/UserProviderTest.java Thu Oct 25 09:13:00 2012
@@ -24,7 +24,8 @@ import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.api.Root;
 import org.apache.jackrabbit.oak.api.Tree;
-import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHook;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHookProvider;
 import org.apache.jackrabbit.oak.plugins.nodetype.InitialContent;
 import org.apache.jackrabbit.oak.spi.security.ConfigurationParameters;
 import org.apache.jackrabbit.oak.spi.security.user.UserConstants;
@@ -61,7 +62,7 @@ public class UserProviderTest {
     public void setUp() throws Exception {
         root = new Oak()
                 .with(new InitialContent())
-                .with(new PropertyIndexHook())
+                .with(new IndexHookManager(new PropertyIndexHookProvider()))
                 .createRoot();
 
         defaultConfig = new ConfigurationParameters();

Modified: jackrabbit/oak/trunk/oak-jcr/src/main/java/org/apache/jackrabbit/oak/jcr/Jcr.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-jcr/src/main/java/org/apache/jackrabbit/oak/jcr/Jcr.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-jcr/src/main/java/org/apache/jackrabbit/oak/jcr/Jcr.java (original)
+++ jackrabbit/oak/trunk/oak-jcr/src/main/java/org/apache/jackrabbit/oak/jcr/Jcr.java Thu Oct 25 09:13:00 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.jackrabbit.oak.jcr;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 
@@ -26,10 +28,11 @@ import org.apache.jackrabbit.mk.api.Micr
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.plugins.commit.AnnotatingConflictHandler;
 import org.apache.jackrabbit.oak.plugins.commit.ConflictValidatorProvider;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneHook;
+import org.apache.jackrabbit.oak.plugins.index.CompositeIndexHookProvider;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexHookProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexProvider;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneReindexHook;
-import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHook;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHookProvider;
 import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexProvider;
 import org.apache.jackrabbit.oak.plugins.name.NameValidatorProvider;
 import org.apache.jackrabbit.oak.plugins.name.NamespaceValidatorProvider;
@@ -45,9 +48,6 @@ import org.apache.jackrabbit.oak.spi.lif
 import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
 import org.apache.jackrabbit.oak.spi.security.SecurityProvider;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.DEFAULT_INDEX_HOME;
-
 public class Jcr {
 
     private final Oak oak;
@@ -71,13 +71,14 @@ public class Jcr {
         with(new TypeValidatorProvider());
         with(new ConflictValidatorProvider());
 
-        with(new PropertyIndexHook());
-        with(new LuceneReindexHook(DEFAULT_INDEX_HOME));
-        with(new LuceneHook(DEFAULT_INDEX_HOME));
+        with(new IndexHookManager(
+                new CompositeIndexHookProvider(
+                new PropertyIndexHookProvider(), 
+                new LuceneIndexHookProvider())));
         with(new AnnotatingConflictHandler());
 
         with(new PropertyIndexProvider());
-        with(new LuceneIndexProvider(DEFAULT_INDEX_HOME));
+        with(new LuceneIndexProvider());
     }
 
     public Jcr() {

Modified: jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/run/Main.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/run/Main.java?rev=1402036&r1=1402035&r2=1402036&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/run/Main.java (original)
+++ jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/run/Main.java Thu Oct 25 09:13:00 2012
@@ -19,6 +19,7 @@ package org.apache.jackrabbit.oak.run;
 import java.io.InputStream;
 import java.util.Properties;
 import java.util.concurrent.Executors;
+
 import javax.jcr.Repository;
 
 import org.apache.jackrabbit.mk.api.MicroKernel;
@@ -28,9 +29,10 @@ import org.apache.jackrabbit.oak.api.Con
 import org.apache.jackrabbit.oak.http.OakServlet;
 import org.apache.jackrabbit.oak.jcr.RepositoryImpl;
 import org.apache.jackrabbit.oak.plugins.commit.ConflictValidatorProvider;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneHook;
-import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneReindexHook;
-import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHook;
+import org.apache.jackrabbit.oak.plugins.index.CompositeIndexHookProvider;
+import org.apache.jackrabbit.oak.plugins.index.IndexHookManager;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexHookProvider;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexHookProvider;
 import org.apache.jackrabbit.oak.plugins.name.NameValidatorProvider;
 import org.apache.jackrabbit.oak.plugins.name.NamespaceValidatorProvider;
 import org.apache.jackrabbit.oak.plugins.nodetype.DefaultTypeEditor;
@@ -48,8 +50,6 @@ import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 
-import static org.apache.jackrabbit.oak.plugins.index.IndexUtils.DEFAULT_INDEX_HOME;
-
 public class Main {
 
     public static final int PORT = 8080;
@@ -204,9 +204,10 @@ public class Main {
             return new CompositeHook(
                     new DefaultTypeEditor(),
                     new ValidatingHook(createDefaultValidatorProvider()),
-                    new PropertyIndexHook(),
-                    new LuceneReindexHook(DEFAULT_INDEX_HOME),
-                    new LuceneHook(DEFAULT_INDEX_HOME));
+                    new IndexHookManager(
+                            new CompositeIndexHookProvider(
+                            new PropertyIndexHookProvider(), 
+                            new LuceneIndexHookProvider())));
         }
 
         private static ValidatorProvider createDefaultValidatorProvider() {