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/04/04 13:37:26 UTC

svn commit: r1464465 - 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/p2/ oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/ oak-l...

Author: alexparvulescu
Date: Thu Apr  4 11:37:25 2013
New Revision: 1464465

URL: http://svn.apache.org/r1464465
Log:
OAK-734 Refactor indexing code to use Editors - Property2Index apply changes directly to NodeBuilder

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHook.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookUpdate.java   (with props)
Removed:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexDiff.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexUpdate.java
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerDiff.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2Index.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookProvider.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexLookup.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexProvider.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexTest.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexDiff.java
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexDiff.java
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexHook.java

Modified: 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=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHook.java Thu Apr  4 11:37:25 2013
@@ -18,25 +18,25 @@ package org.apache.jackrabbit.oak.plugin
 
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.spi.commit.Editor;
-import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
 
 /**
  * Represents the content of a QueryIndex as well as a mechanism for keeping
  * this content up to date.
- * <p>
+ * <br>
  * An IndexHook listens for changes to the content and updates the index data
  * accordingly.
  */
 public interface IndexHook extends Editor {
 
     /**
-     * Re-create this index.
+     * Re-create this index using the given state
      * 
      * @param state
      *            the parent of the node "oak:index" (the node that contains the
      *            index definition)
      * @throws CommitFailedException
      */
-    void reindex(NodeBuilder state) throws CommitFailedException;
+    void reindex(NodeState state) throws CommitFailedException;
 
 }

Modified: 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=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManager.java Thu Apr  4 11:37:25 2013
@@ -24,10 +24,8 @@ import org.apache.jackrabbit.oak.spi.sta
 
 /**
  * Keeps existing IndexHooks updated.
- * 
- * <p>
+ * <br>
  * The existing index list is obtained via the IndexHookProvider.
- * </p>
  * 
  * @see IndexHook
  * @see IndexHookProvider

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerDiff.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerDiff.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerDiff.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/IndexHookManagerDiff.java Thu Apr  4 11:37:25 2013
@@ -40,9 +40,8 @@ import org.apache.jackrabbit.oak.spi.sta
 import com.google.common.collect.Lists;
 
 /**
- * Acts as a composite NodeStateDiff, it delegates all the diff's events to the
- * existing IndexHooks.
- * 
+ * Acts as a composite Editor, it delegates all the diff's events to the
+ * existing IndexHooks. <br>
  * This allows for a simultaneous update of all the indexes via a single
  * traversal of the changes.
  */
@@ -62,13 +61,12 @@ class IndexHookManagerDiff implements Ed
     @Override
     public void enter(NodeState before, NodeState after)
             throws CommitFailedException {
-        if (node != null && node.hasChildNode(INDEX_DEFINITIONS_NAME)) {
+        if (after != null && after.hasChildNode(INDEX_DEFINITIONS_NAME)) {
             Set<String> existingTypes = new HashSet<String>();
             Set<String> reindexTypes = new HashSet<String>();
-
-            NodeBuilder index = node.child(INDEX_DEFINITIONS_NAME);
+            NodeState index = after.getChildNode(INDEX_DEFINITIONS_NAME);
             for (String indexName : index.getChildNodeNames()) {
-                NodeBuilder indexChild = index.child(indexName);
+                NodeState indexChild = index.getChildNode(indexName);
                 if (isIndexNodeType(indexChild.getProperty(JCR_PRIMARYTYPE))) {
                     PropertyState reindexPS = indexChild
                             .getProperty(REINDEX_PROPERTY_NAME);
@@ -105,7 +103,7 @@ class IndexHookManagerDiff implements Ed
                 this.inner = new CompositeEditor(hooks);
                 this.inner.enter(before, after);
                 for (IndexHook ih : reindex) {
-                    ih.reindex(node);
+                    ih.reindex(after);
                 }
             }
         }

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2Index.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2Index.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2Index.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2Index.java Thu Apr  4 11:37:25 2013
@@ -37,7 +37,7 @@ import com.google.common.base.Charsets;
  * 
  * <p>
  * To define a property index on a subtree you have to add an <code>oak:index</code> node.
- * 
+ * <br>
  * Next (as a child node) follows the index definition node that:
  * <ul>
  * <li>must be of type <code>oak:queryIndexDefinition</code></li>
@@ -53,11 +53,11 @@ import com.google.common.base.Charsets;
  * </ul>
  * </p>
  * <p>
- * Note: <code>propertyNames</code> can be a list of properties, and it is optional.in case it is missing, the node name will be used as a property name reference value
- * </p>
- * 
- * <p>
- * Note: <code>reindex</code> is a property that when set to <code>true</code>, triggers a full content reindex.
+ * Notes:
+ * <ul>
+ * <li> <code>propertyNames</code> can be a list of properties, and it is optional.in case it is missing, the node name will be used as a property name reference value</li>
+ * <li> <code>reindex</code> is a property that when set to <code>true</code>, triggers a full content reindex.</li>
+ * </ul>
  * </p>
  * 
  * <pre>

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHook.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHook.java?rev=1464465&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHook.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHook.java Thu Apr  4 11:37:25 2013
@@ -0,0 +1,315 @@
+/*
+ * 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.p2;
+
+import static com.google.common.collect.Lists.newArrayList;
+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.p2.Property2Index.TYPE;
+import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+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.index.IndexHook;
+import org.apache.jackrabbit.oak.plugins.index.p2.strategy.ContentMirrorStoreStrategy;
+import org.apache.jackrabbit.oak.plugins.index.p2.strategy.IndexStoreStrategy;
+import org.apache.jackrabbit.oak.spi.commit.Editor;
+import org.apache.jackrabbit.oak.spi.commit.EditorHook;
+import org.apache.jackrabbit.oak.spi.commit.EditorProvider;
+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 com.google.common.collect.ImmutableList;
+
+/**
+ * {@link IndexHook} implementation that is responsible for keeping the
+ * {@link Property2Index} up to date.
+ * <br>
+ * There is a tree of PropertyIndexDiff objects, each object represents the
+ * changes at a given node.
+ * 
+ * @see Property2Index
+ * @see Property2IndexLookup
+ */
+class Property2IndexHook implements IndexHook, Closeable {
+
+    protected static String propertyNames = "propertyNames";
+
+    protected static String declaringNodeTypes = "declaringNodeTypes";
+
+    private final IndexStoreStrategy store = new ContentMirrorStoreStrategy();
+
+    /**
+     * The parent (null if this is the root node).
+     */
+    private final Property2IndexHook parent;
+
+    /**
+     * The node (never null).
+     */
+    private final NodeBuilder node;
+
+    /**
+     * The node name (the path element). Null for the root node.
+     */
+    private final String nodeName;
+
+    /**
+     * The path of the changed node (built lazily).
+     */
+    private String path;
+
+    /**
+     * The map of known indexes. Key: the property name. Value: the list of
+     * indexes (it is possible to have multiple indexes for the same property
+     * name).
+     */
+    private final Map<String, List<Property2IndexHookUpdate>> indexMap;
+
+    public Property2IndexHook(NodeBuilder root) {
+        this(null, root, null, "/",
+                new HashMap<String, List<Property2IndexHookUpdate>>());
+    }
+
+    private Property2IndexHook(Property2IndexHook parent, String nodeName) {
+        this(parent, getChildNode(parent.node, nodeName), nodeName, null,
+                parent.indexMap);
+    }
+
+    private Property2IndexHook(Property2IndexHook parent, NodeBuilder node,
+            String nodeName, String path,
+            Map<String, List<Property2IndexHookUpdate>> indexMap) {
+        this.parent = parent;
+        this.node = node;
+        this.nodeName = nodeName;
+        this.path = path;
+        this.indexMap = indexMap;
+    }
+
+    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) {
+            path = concat(parent.getPath(), nodeName);
+        }
+        return path;
+    }
+
+    /**
+     * Get all the indexes for the given property name.
+     * 
+     * @param propertyName
+     *            the property name
+     * @return the indexes
+     */
+    private Iterable<Property2IndexHookUpdate> getIndexes(String propertyName) {
+        List<Property2IndexHookUpdate> indexes = indexMap.get(propertyName);
+        if (indexes == null) {
+            return ImmutableList.of();
+        }
+        List<Property2IndexHookUpdate> filtered = new ArrayList<Property2IndexHookUpdate>();
+        for (Property2IndexHookUpdate pi : indexes) {
+            if (node == null || pi.getNodeTypeNames() == null
+                    || pi.getNodeTypeNames().isEmpty()) {
+                filtered.add(pi);
+                continue;
+            }
+            PropertyState ps = node.getProperty(JCR_PRIMARYTYPE);
+            String type = ps != null && !ps.isArray() ? ps
+                    .getValue(Type.STRING) : null;
+            if (type != null) {
+                for (String typeName : pi.getNodeTypeNames()) {
+                    if (typeName.equals(type)) {
+                        filtered.add(pi);
+                        break;
+                    }
+                }
+            }
+        }
+        return filtered;
+    }
+
+    /**
+     * Add the index definitions to the in-memory set of known index
+     * definitions.
+     * 
+     * @param state
+     *            the node state that contains the index definition
+     * @param indexName
+     *            the name of the index
+     */
+    private void addIndexes(NodeState state, String indexName) {
+        List<String> typeNames = ImmutableList.of();
+        PropertyState appliesTo = state.getProperty(declaringNodeTypes);
+        if (appliesTo != null) {
+            typeNames = newArrayList(appliesTo.getValue(Type.STRINGS));
+            Collections.sort(typeNames);
+        }
+        PropertyState ps = state.getProperty(propertyNames);
+
+        Iterable<String> propertyNames = ps != null ? ps.getValue(Type.STRINGS)
+                : ImmutableList.of(indexName);
+        for (String pname : propertyNames) {
+            List<Property2IndexHookUpdate> list = this.indexMap.get(pname);
+            if (list == null) {
+                list = newArrayList();
+                this.indexMap.put(pname, list);
+            }
+            boolean exists = false;
+            String localPath = getPath();
+            for (Property2IndexHookUpdate piu : list) {
+                if (localPath.equals(piu.getPath())
+                        && typeNames.equals(piu.getNodeTypeNames())) {
+                    exists = true;
+                    break;
+                }
+            }
+            if (!exists) {
+                list.add(new Property2IndexHookUpdate(getPath(), node.child(
+                        INDEX_DEFINITIONS_NAME).child(indexName), store,
+                        typeNames));
+            }
+        }
+    }
+
+    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 {
+        if (after != null && after.hasChildNode(INDEX_DEFINITIONS_NAME)) {
+            NodeState index = after.getChildNode(INDEX_DEFINITIONS_NAME);
+            for (String indexName : index.getChildNodeNames()) {
+                NodeState child = index.getChildNode(indexName);
+                if (isIndexNode(child)) {
+                    addIndexes(child, indexName);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void leave(NodeState before, NodeState after)
+            throws CommitFailedException {
+    }
+
+    @Override
+    public void propertyAdded(PropertyState after) throws CommitFailedException {
+        for (Property2IndexHookUpdate update : getIndexes(after.getName())) {
+            update.insert(getPath(), after);
+        }
+    }
+
+    @Override
+    public void propertyChanged(PropertyState before, PropertyState after)
+            throws CommitFailedException {
+        for (Property2IndexHookUpdate update : getIndexes(after.getName())) {
+            update.remove(getPath(), before);
+            update.insert(getPath(), after);
+        }
+    }
+
+    @Override
+    public void propertyDeleted(PropertyState before)
+            throws CommitFailedException {
+        for (Property2IndexHookUpdate update : getIndexes(before.getName())) {
+            update.remove(getPath(), before);
+        }
+    }
+
+    @Override
+    public Editor childNodeAdded(String name, NodeState after)
+            throws CommitFailedException {
+        return childNodeChanged(name, EMPTY_NODE, after);
+    }
+
+    @Override
+    public Editor childNodeChanged(String name, NodeState before,
+            NodeState after) throws CommitFailedException {
+        if (NodeStateUtils.isHidden(name)) {
+            return null;
+        }
+        return new Property2IndexHook(this, name);
+    }
+
+    @Override
+    public Editor childNodeDeleted(String name, NodeState before)
+            throws CommitFailedException {
+        return childNodeChanged(name, before, EMPTY_NODE);
+    }
+
+    @Override
+    public void reindex(NodeState state) throws CommitFailedException {
+        boolean reindex = false;
+        for (List<Property2IndexHookUpdate> updateList : indexMap.values()) {
+            for (Property2IndexHookUpdate update : updateList) {
+                if (update.getAndResetReindexFlag()) {
+                    reindex = true;
+                }
+            }
+        }
+        if (reindex) {
+            EditorProvider provider = new EditorProvider() {
+                @Override
+                public Editor getRootEditor(NodeState before, NodeState after,
+                        NodeBuilder builder) {
+                    return new Property2IndexHook(node);
+                }
+            };
+            EditorHook eh = new EditorHook(provider);
+            eh.processCommit(EMPTY_NODE, state);
+        }
+    }
+
+    // -----------------------------------------------------< Closeable >--
+
+    @Override
+    public void close() throws IOException {
+        indexMap.clear();
+    }
+}

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

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

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookProvider.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookProvider.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookProvider.java Thu Apr  4 11:37:25 2013
@@ -31,7 +31,7 @@ import com.google.common.collect.Immutab
 /**
  * Service that provides PropertyIndex based IndexHooks.
  * 
- * @see Property2IndexDiff
+ * @see Property2IndexHook
  * @see IndexHookProvider
  * 
  */
@@ -43,7 +43,7 @@ public class Property2IndexHookProvider 
     public List<? extends IndexHook> getIndexHooks(String type,
             NodeBuilder builder) {
         if (TYPE.equals(type)) {
-            return ImmutableList.of(new Property2IndexDiff(builder));
+            return ImmutableList.of(new Property2IndexHook(builder));
         }
         return ImmutableList.of();
     }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookUpdate.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookUpdate.java?rev=1464465&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookUpdate.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexHookUpdate.java Thu Apr  4 11:37:25 2013
@@ -0,0 +1,140 @@
+/*
+ * 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.p2;
+
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.p2.Property2Index.encode;
+
+import java.util.List;
+
+import javax.jcr.PropertyType;
+
+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.p2.strategy.IndexStoreStrategy;
+import org.apache.jackrabbit.oak.spi.query.PropertyValues;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Takes care of applying the updates to the index content.
+ */
+class Property2IndexHookUpdate {
+
+    private final IndexStoreStrategy store;
+
+    /**
+     * The path of the index definition (where the index data is stored).
+     */
+    private final String path;
+
+    /**
+     * The node types that this index applies to. If <code>null</code> or
+     * <code>empty</code> then the node type of the indexed node is ignored
+     * 
+     */
+    private final List<String> nodeTypeNames;
+
+    /**
+     * The node where the index definition is stored.
+     */
+    private final NodeBuilder node;
+
+    /**
+     * The node where the index content is stored.
+     */
+    private final NodeBuilder index;
+
+    private final boolean unique;
+
+    public Property2IndexHookUpdate(String path, NodeBuilder node,
+            IndexStoreStrategy store, List<String> nodeTypeNames) {
+        this.path = path;
+        this.node = node;
+        this.store = store;
+        this.nodeTypeNames = nodeTypeNames;
+        index = this.node.child(":index");
+        PropertyState uniquePS = node.getProperty("unique");
+        unique = uniquePS != null && !uniquePS.isArray()
+                && uniquePS.getValue(Type.BOOLEAN);
+    }
+
+    String getPath() {
+        return path;
+    }
+
+    List<String> getNodeTypeNames() {
+        return nodeTypeNames;
+    }
+
+    /**
+     * A property value was added at the given path.
+     * 
+     * @param path
+     *            the path
+     * @param value
+     *            the value
+     */
+    void insert(String path, PropertyState value) throws CommitFailedException {
+        Preconditions.checkArgument(path.startsWith(this.path));
+        if (value.getType().tag() == PropertyType.BINARY) {
+            return;
+        }
+        for (String key : encode(PropertyValues.create(value))) {
+            store.insert(index, key, unique, ImmutableSet.of(trimm(path)));
+        }
+    }
+
+    /**
+     * A property value was removed at the given path.
+     * 
+     * @param path
+     *            the path
+     * @param value
+     *            the value
+     */
+    public void remove(String path, PropertyState value)
+            throws CommitFailedException {
+        Preconditions.checkArgument(path.startsWith(this.path));
+        if (value.getType().tag() == PropertyType.BINARY) {
+            return;
+        }
+        for (String key : encode(PropertyValues.create(value))) {
+            store.remove(index, key, ImmutableSet.of(trimm(path)));
+        }
+    }
+
+    private String trimm(String path) {
+        path = path.substring(this.path.length());
+        if ("".equals(path)) {
+            return "/";
+        }
+        return path;
+    }
+
+    boolean getAndResetReindexFlag() {
+        PropertyState reindexPS = node.getProperty(REINDEX_PROPERTY_NAME);
+        boolean reindex = reindexPS == null
+                || (reindexPS != null && reindexPS.getValue(Type.BOOLEAN));
+        node.setProperty(REINDEX_PROPERTY_NAME, false);
+        return reindex;
+    }
+
+}

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

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

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexLookup.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexLookup.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexLookup.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexLookup.java Thu Apr  4 11:37:25 2013
@@ -38,11 +38,9 @@ import org.apache.jackrabbit.oak.spi.sta
 
 /**
  * Is responsible for querying the property index content.
- * 
- * <p>
+ * <br>
  * This class can be used directly on a subtree where there is an index defined
  * by supplying a {@link NodeState} root.
- * </p>
  * 
  * <pre>
  * <code>

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexProvider.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexProvider.java Thu Apr  4 11:37:25 2013
@@ -28,7 +28,7 @@ import com.google.common.collect.Immutab
 
 /**
  * A provider for property indexes.
- * <p>
+ * <br>
  * Even if there are multiple index definitions, there is only actually one
  * PropertyIndex instance, which is used for all indexes.
  * 

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexTest.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/p2/Property2IndexTest.java Thu Apr  4 11:37:25 2013
@@ -74,7 +74,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -138,7 +138,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -203,7 +203,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -267,7 +267,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -316,7 +316,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -342,7 +342,7 @@ public class Property2IndexTest {
                 .setProperty("unique", "true")
                 .setProperty("propertyNames", Arrays.asList("foo"),
                         Type.STRINGS)
-                .setProperty(Property2IndexDiff.declaringNodeTypes,
+                .setProperty(Property2IndexHook.declaringNodeTypes,
                         Arrays.asList("typeFoo"), Type.STRINGS);
         NodeState before = builder.getNodeState();
         builder = before.builder();
@@ -356,7 +356,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -377,7 +377,7 @@ public class Property2IndexTest {
                 .setProperty("unique", "true")
                 .setProperty("propertyNames", Arrays.asList("foo"),
                         Type.STRINGS)
-                .setProperty(Property2IndexDiff.declaringNodeTypes,
+                .setProperty(Property2IndexHook.declaringNodeTypes,
                         Arrays.asList("typeFoo"), Type.STRINGS);
         NodeState before = builder.getNodeState();
         builder = before.builder();
@@ -391,7 +391,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);
@@ -417,7 +417,7 @@ public class Property2IndexTest {
                 .setProperty("unique", "true")
                 .setProperty("propertyNames", Arrays.asList("foo"),
                         Type.STRINGS)
-                .setProperty(Property2IndexDiff.declaringNodeTypes,
+                .setProperty(Property2IndexHook.declaringNodeTypes,
                         Arrays.asList("typeFoo"), Type.STRINGS);
         builder.child("a").setProperty("jcr:primaryType", "typeFoo", Type.NAME)
                 .setProperty("foo", "abc");
@@ -432,7 +432,7 @@ public class Property2IndexTest {
             @Override
             public Editor getRootEditor(NodeState before, NodeState after,
                     NodeBuilder builder) {
-                return new Property2IndexDiff(builder);
+                return new Property2IndexHook(builder);
             }
         };
         EditorHook hook = new EditorHook(provider);

Modified: 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/LuceneIndexDiff.java?rev=1464465&r1=1464464&r2=1464465&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/LuceneIndexDiff.java Thu Apr  4 11:37:25 2013
@@ -206,7 +206,7 @@ public class LuceneIndexDiff implements 
     }
 
     @Override
-    public void reindex(NodeBuilder state) throws CommitFailedException {
+    public void reindex(NodeState state) throws CommitFailedException {
         boolean reindex = false;
         for (LuceneIndexUpdate update : updates.values()) {
             if (update.getAndResetReindexFlag()) {
@@ -222,7 +222,7 @@ public class LuceneIndexDiff implements 
                 }
             };
             EditorHook eh = new EditorHook(provider);
-            eh.processCommit(EMPTY_NODE, state.getNodeState());
+            eh.processCommit(EMPTY_NODE, state);
         }
     }
 

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexDiff.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexDiff.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexDiff.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexDiff.java Thu Apr  4 11:37:25 2013
@@ -204,7 +204,7 @@ public class SolrIndexDiff implements In
     }
 
     @Override
-    public void reindex(NodeBuilder state) throws CommitFailedException {
+    public void reindex(NodeState state) throws CommitFailedException {
         boolean reindex = false;
         for (SolrIndexUpdate update : updates.values()) {
             if (update.getAndResetReindexFlag()) {
@@ -220,7 +220,7 @@ public class SolrIndexDiff implements In
                 }
             };
             EditorHook eh = new EditorHook(provider);
-            eh.processCommit(EMPTY_NODE, state.getNodeState());
+            eh.processCommit(EMPTY_NODE, state);
         }
     }
 

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexHook.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexHook.java?rev=1464465&r1=1464464&r2=1464465&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexHook.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexHook.java Thu Apr  4 11:37:25 2013
@@ -235,11 +235,11 @@ public class SolrIndexHook implements In
     }
 
     @Override
-    public void reindex(NodeBuilder state) throws CommitFailedException {
+    public void reindex(NodeState state) throws CommitFailedException {
         try {
             close();
             deleteByIdQueryBuilder.append(getPath()).append("*");
-            solrInputDocuments.addAll(docsFromState(getPath(), state.getNodeState()));
+            solrInputDocuments.addAll(docsFromState(getPath(), state));
             apply();
         } catch (IOException e) {
             throw new CommitFailedException(e);