You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by ch...@apache.org on 2014/10/30 18:54:42 UTC

svn commit: r1635563 - in /jackrabbit/oak/trunk/oak-lucene/src: main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ test/java/org/apache/jackrabbit/oak/plugins/index/lucene/

Author: chetanm
Date: Thu Oct 30 17:54:42 2014
New Revision: 1635563

URL: http://svn.apache.org/r1635563
Log:
OAK-2240 - Support for indexing relative properties

Applying updated patch by Alex

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativeProperty.java   (with props)
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativePropertyTest.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndex.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinitionTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndexTest.java

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinition.java Thu Oct 30 17:54:42 2014
@@ -19,6 +19,7 @@
 
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -29,8 +30,6 @@ import javax.jcr.PropertyType;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
@@ -40,6 +39,8 @@ import org.apache.lucene.codecs.Codec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.collect.Maps.newHashMap;
+import static com.google.common.collect.Sets.newHashSet;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.DECLARING_NODE_TYPES;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.ENTRY_COUNT_PROPERTY_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_COUNT;
@@ -89,6 +90,10 @@ class IndexDefinition {
 
     private final Codec codec;
 
+    private final Map<String,RelativeProperty> relativeProps;
+
+    private final Set<String> relativePropNames;
+
     /**
      * Defines the maximum estimated entry count configured.
      * Defaults to {#DEFAULT_ENTRY_COUNT}
@@ -122,15 +127,10 @@ class IndexDefinition {
         this.fullTextEnabled = getOptionalValue(defn, FULL_TEXT_ENABLED, true);
         //Storage is disabled for non full text indexes
         this.storageEnabled = this.fullTextEnabled && getOptionalValue(defn, EXPERIMENTAL_STORAGE, true);
-
-        Map<String, PropertyDefinition> propDefns = Maps.newHashMap();
-        NodeBuilder propNode = defn.getChildNode(LuceneIndexConstants.PROP_NODE);
-        for(String propName : Iterables.concat(includes, orderedProps)){
-            if(propNode.hasChildNode(propName)){
-                propDefns.put(propName, new PropertyDefinition(this, propName, propNode.child(propName)));
-            }
-        }
-        this.propDefns = ImmutableMap.copyOf(propDefns);
+        //TODO Flag out invalid propertyNames like one which are absolute
+        this.relativeProps = collectRelativeProps(Iterables.concat(includes, orderedProps));
+        this.propDefns = collectPropertyDefns(defn);
+        this.relativePropNames = collectRelPropertyNames(this.relativeProps.values());
 
         String functionName = getOptionalValue(defn, LuceneIndexConstants.FUNC_NAME, null);
         this.funcName = functionName != null ? "native*" + functionName : null;
@@ -237,8 +237,74 @@ class IndexDefinition {
         return entryCount;
     }
 
+    public Collection<RelativeProperty> getRelativeProps() {
+        return relativeProps.values();
+    }
+
+    /**
+     * Collects the relative properties where the property name matches given name. Note
+     * that multiple relative properties can end with same name e.g. foo/bar, baz/bar
+     *
+     * @param name property name without path
+     * @param relProps matching relative properties where the relative property path ends
+     *                 with given name
+     */
+    public void collectRelPropsForName(String name, Collection<RelativeProperty> relProps){
+        if(hasRelativeProperty(name)){
+            for(RelativeProperty rp : relativeProps.values()){
+                if(rp.name.equals(name)){
+                    relProps.add(rp);
+                }
+            }
+        }
+    }
+
+    boolean hasRelativeProperties(){
+        return !relativePropNames.isEmpty();
+    }
+
+    boolean hasRelativeProperty(String name) {
+        return relativePropNames.contains(name);
+    }
+
     //~------------------------------------------< Internal >
 
+    private Map<String, RelativeProperty> collectRelativeProps(Iterable<String> propNames) {
+        Map<String, RelativeProperty> relProps = newHashMap();
+        for (String propName : propNames) {
+            if (RelativeProperty.isRelativeProperty(propName)) {
+                relProps.put(propName, new RelativeProperty(propName));
+            }
+        }
+        return ImmutableMap.copyOf(relProps);
+    }
+
+    private Map<String, PropertyDefinition> collectPropertyDefns(NodeBuilder defn) {
+        Map<String, PropertyDefinition> propDefns = newHashMap();
+        NodeBuilder propNode = defn.getChildNode(LuceneIndexConstants.PROP_NODE);
+        for (String propName : Iterables.concat(includes, orderedProps)) {
+            NodeBuilder propDefnNode;
+            if (relativeProps.containsKey(propName)) {
+                propDefnNode = relativeProps.get(propName).getPropDefnNode(propNode);
+            } else {
+                propDefnNode = propNode.getChildNode(propName);
+            }
+
+            if (propDefnNode.exists()) {
+                propDefns.put(propName, new PropertyDefinition(this, propName, propDefnNode));
+            }
+        }
+        return ImmutableMap.copyOf(propDefns);
+    }
+
+    private Set<String> collectRelPropertyNames(Collection<RelativeProperty> props) {
+        Set<String> propNames = newHashSet();
+        for (RelativeProperty prop : props){
+            propNames.add(prop.name);
+        }
+        return ImmutableSet.copyOf(propNames);
+    }
+
     private Codec createCodec() {
         String codecName = getOptionalValue(definition, LuceneIndexConstants.CODEC_NAME, null);
         Codec codec = null;
@@ -271,10 +337,10 @@ class IndexDefinition {
     }
 
     private static Set<String> toLowerCase(Set<String> values){
-        Set<String> result = Sets.newHashSet();
+        Set<String> result = newHashSet();
         for(String val : values){
             result.add(val.toLowerCase());
         }
-        return Collections.unmodifiableSet(result);
+        return ImmutableSet.copyOf(result);
     }
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java Thu Oct 30 17:54:42 2014
@@ -31,6 +31,8 @@ import java.util.List;
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
+import com.google.common.collect.Lists;
+
 import org.apache.jackrabbit.JcrConstants;
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
@@ -89,6 +91,13 @@ public class LuceneIndexEditor implement
 
     private final Predicate<NodeState> typePredicate;
 
+    private List<RelativeProperty> changedRelativeProps;
+
+    /**
+     * Flag indicating if the current tree being traversed has a deleted parent.
+     */
+    private final boolean isDeleted;
+
     LuceneIndexEditor(NodeState root, NodeBuilder definition, Analyzer analyzer,
         IndexUpdateCallback updateCallback) throws CommitFailedException {
         this.parent = null;
@@ -102,15 +111,17 @@ public class LuceneIndexEditor implement
         } else {
             typePredicate = Predicates.alwaysTrue();
         }
+        this.isDeleted = false;
     }
 
-    private LuceneIndexEditor(LuceneIndexEditor parent, String name) {
+    private LuceneIndexEditor(LuceneIndexEditor parent, String name, boolean isDeleted) {
         this.parent = parent;
         this.name = name;
         this.path = null;
         this.context = parent.context;
         this.root = parent.root;
         this.typePredicate = parent.typePredicate;
+        this.isDeleted = isDeleted;
     }
 
     public String getPath() {
@@ -141,6 +152,10 @@ public class LuceneIndexEditor implement
             }
         }
 
+        if (changedRelativeProps != null) {
+            markParentsOnRelPropChange();
+        }
+
         if (parent == null) {
             try {
                 context.closeWriter();
@@ -156,47 +171,57 @@ public class LuceneIndexEditor implement
 
     @Override
     public void propertyAdded(PropertyState after) {
-        propertiesChanged = true;
+        markPropertiesChanged();
+        checkForRelativePropertyChange(after.getName());
     }
 
     @Override
     public void propertyChanged(PropertyState before, PropertyState after) {
-        propertiesChanged = true;
+        markPropertiesChanged();
+        checkForRelativePropertyChange(before.getName());
     }
 
     @Override
     public void propertyDeleted(PropertyState before) {
-        propertiesChanged = true;
+        markPropertiesChanged();
+        checkForRelativePropertyChange(before.getName());
     }
 
     @Override
     public Editor childNodeAdded(String name, NodeState after) {
-        return new LuceneIndexEditor(this, name);
+        return new LuceneIndexEditor(this, name, false);
     }
 
     @Override
     public Editor childNodeChanged(
             String name, NodeState before, NodeState after) {
-        return new LuceneIndexEditor(this, name);
+        return new LuceneIndexEditor(this, name, false);
     }
 
     @Override
     public Editor childNodeDeleted(String name, NodeState before)
             throws CommitFailedException {
-        String path = concat(getPath(), name);
 
-        try {
-            IndexWriter writer = context.getWriter();
-            // Remove all index entries in the removed subtree
-            writer.deleteDocuments(newPathTerm(path));
-            writer.deleteDocuments(new PrefixQuery(newPathTerm(path + "/")));
-            this.context.indexUpdate();
-        } catch (IOException e) {
-            throw new CommitFailedException(
-                    "Lucene", 5, "Failed to remove the index entries of"
-                    + " the removed subtree " + path, e);
+        if (!isDeleted) {
+            // tree deletion is handled on the parent node
+            String path = concat(getPath(), name);
+            try {
+                IndexWriter writer = context.getWriter();
+                // Remove all index entries in the removed subtree
+                writer.deleteDocuments(newPathTerm(path));
+                writer.deleteDocuments(new PrefixQuery(newPathTerm(path + "/")));
+                this.context.indexUpdate();
+            } catch (IOException e) {
+                throw new CommitFailedException("Lucene", 5,
+                        "Failed to remove the index entries of"
+                                + " the removed subtree " + path, e);
+            }
         }
 
+        if (context.getDefinition().hasRelativeProperties()) {
+            // need to possibly update aggregated properties
+            return new LuceneIndexEditor(this, name, true);
+        }
         return null; // no need to recurse down the removed subtree
     }
 
@@ -237,7 +262,7 @@ public class LuceneIndexEditor implement
             }
 
             if (context.getDefinition().isOrdered(pname)) {
-                dirty |= addTypedOrderedFields(fields, property);
+                dirty |= addTypedOrderedFields(fields, property, pname);
             }
 
             if (context.includeProperty(pname)) {
@@ -255,6 +280,8 @@ public class LuceneIndexEditor implement
             }
         }
 
+        dirty |= indexRelativeProperties(path, fields, state);
+
         if (isUpdate && !dirty) {
             // updated the state but had no relevant changes
             return null;
@@ -288,7 +315,7 @@ public class LuceneIndexEditor implement
             return true;
         } else if(!context.isFullTextEnabled()
                 && FieldFactory.canCreateTypedField(property.getType())){
-            return addTypedFields(fields, property);
+            return addTypedFields(fields, property, pname);
         } else {
             boolean dirty = false;
             for (String value : property.getValue(Type.STRINGS)) {
@@ -305,21 +332,20 @@ public class LuceneIndexEditor implement
         }
     }
 
-    private boolean addTypedFields(List<Field> fields, PropertyState property) throws CommitFailedException {
+    private boolean addTypedFields(List<Field> fields, PropertyState property, String pname) throws CommitFailedException {
         int tag = property.getType().tag();
-        String name = property.getName();
         boolean fieldAdded = false;
         for (int i = 0; i < property.count(); i++) {
             Field f = null;
             if (tag == Type.LONG.tag()) {
-                f = new LongField(name, property.getValue(Type.LONG, i), Field.Store.NO);
+                f = new LongField(pname, property.getValue(Type.LONG, i), Field.Store.NO);
             } else if (tag == Type.DATE.tag()) {
                 String date = property.getValue(Type.DATE, i);
-                f = new LongField(name, FieldFactory.dateToLong(date), Field.Store.NO);
+                f = new LongField(pname, FieldFactory.dateToLong(date), Field.Store.NO);
             } else if (tag == Type.DOUBLE.tag()) {
-                f = new DoubleField(name, property.getValue(Type.DOUBLE, i), Field.Store.NO);
+                f = new DoubleField(pname, property.getValue(Type.DOUBLE, i), Field.Store.NO);
             } else if (tag == Type.BOOLEAN.tag()) {
-                f = new StringField(name, property.getValue(Type.BOOLEAN, i).toString(), Field.Store.NO);
+                f = new StringField(pname, property.getValue(Type.BOOLEAN, i).toString(), Field.Store.NO);
             }
 
             if (f != null) {
@@ -331,10 +357,10 @@ public class LuceneIndexEditor implement
         return fieldAdded;
     }
 
-    private boolean addTypedOrderedFields(List<Field> fields, PropertyState property) throws CommitFailedException {
+    private boolean addTypedOrderedFields(List<Field> fields, PropertyState property, String pname) throws CommitFailedException {
         int tag = property.getType().tag();
 
-        int idxDefinedTag = getIndexDefinitionType(property);
+        int idxDefinedTag = getIndexDefinitionType(pname);
         // Try converting type to the defined type in the index definition
         if (tag != idxDefinedTag) {
             log.debug(
@@ -345,7 +371,7 @@ public class LuceneIndexEditor implement
             tag = idxDefinedTag;
         }
 
-        String name = FieldNames.createDocValFieldName(property.getName());
+        String name = FieldNames.createDocValFieldName(pname);
         boolean fieldAdded = false;
         for (int i = 0; i < property.count(); i++) {
             Field f = null;
@@ -376,20 +402,23 @@ public class LuceneIndexEditor implement
                 log.warn(
                     "Ignoring ordered property. Could not convert property {} of type {} to type " +
                         "{} for path {}",
-                    property, Type.fromTag(property.getType().tag(), false),
+                    pname, Type.fromTag(property.getType().tag(), false),
                     Type.fromTag(tag, false), getPath(), e);
             }
         }
         return fieldAdded;
     }
 
-    private int getIndexDefinitionType(PropertyState property) {
-        int idxDefinedTag =
-            context.getDefinition().getPropDefn(property.getName()).getPropertyType();
-        if (idxDefinedTag == Type.UNDEFINED.tag()) {
-            idxDefinedTag = Type.STRING.tag();
+    private int getIndexDefinitionType(String pname) {
+        int type = Type.UNDEFINED.tag();
+        if (context.getDefinition().hasPropertyDefinition(pname)) {
+            type = context.getDefinition().getPropDefn(pname).getPropertyType();
+        }
+        if (type == Type.UNDEFINED.tag()) {
+            //If no explicit type is defined we assume it to be string
+            type = Type.STRING.tag();
         }
-        return idxDefinedTag;
+        return type;
     }
 
     private static boolean isVisible(String name) {
@@ -417,6 +446,61 @@ public class LuceneIndexEditor implement
         return fields;
     }
 
+    private boolean indexRelativeProperties(String path, List<Field> fields, NodeState state) throws CommitFailedException {
+        IndexDefinition defn = context.getDefinition();
+        boolean dirty = false;
+        for (RelativeProperty rp : defn.getRelativeProps()){
+            String pname = rp.propertyPath;
+
+            PropertyState property = rp.getProperty(state);
+
+            if (property == null){
+                continue;
+            }
+
+            if (defn.isOrdered(pname)) {
+                dirty |= addTypedOrderedFields(fields, property, pname);
+            }
+
+            dirty |= indexProperty(path, fields, state, property, pname);
+        }
+        return dirty;
+    }
+
+    private void checkForRelativePropertyChange(String name) {
+        if (context.getDefinition().hasRelativeProperty(name)) {
+            context.getDefinition().collectRelPropsForName(name, getChangedRelProps());
+        }
+    }
+
+    private void markParentsOnRelPropChange() {
+        for (RelativeProperty rp : changedRelativeProps) {
+            LuceneIndexEditor p = this;
+            for (String parentName : rp.ancestors) {
+                if (p == null || !p.name.equals(parentName)) {
+                    p = null;
+                    break;
+                }
+                p = p.parent;
+            }
+
+            if (p != null) {
+                p.markPropertiesChanged();
+            }
+        }
+    }
+
+    private List<RelativeProperty> getChangedRelProps(){
+        if (changedRelativeProps == null) {
+            changedRelativeProps = Lists.newArrayList();
+        }
+        return changedRelativeProps;
+    }
+
+    private void markPropertiesChanged() {
+        propertiesChanged = true;
+    }
+
     private String parseStringValue(Blob v, Metadata metadata, String path) {
         WriteOutContentHandler handler = new WriteOutContentHandler();
         try {

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndex.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndex.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndex.java Thu Oct 30 17:54:42 2014
@@ -40,7 +40,6 @@ import com.google.common.collect.Sets;
 import org.apache.jackrabbit.oak.api.PropertyValue;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.PathUtils;
-import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.MoreLikeThisHelper;
 import org.apache.jackrabbit.oak.query.QueryEngineSettings;
 import org.apache.jackrabbit.oak.query.QueryImpl;
@@ -843,10 +842,6 @@ public class LucenePropertyIndex impleme
     private static boolean isExcludedProperty(PropertyRestriction pr,
             IndexDefinition definition) {
         String name = pr.propertyName;
-        if (name.contains("/")) {
-            // lucene cannot handle child-level property restrictions
-            return true;
-        }
 
         boolean includeProperty = definition.includeProperty(name);
         // check name

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativeProperty.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativeProperty.java?rev=1635563&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativeProperty.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativeProperty.java Thu Oct 30 17:54:42 2014
@@ -0,0 +1,102 @@
+/*
+ * 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 javax.annotation.CheckForNull;
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.Iterables.toArray;
+import static org.apache.jackrabbit.oak.commons.PathUtils.elements;
+import static org.apache.jackrabbit.oak.commons.PathUtils.isAbsolute;
+
+class RelativeProperty {
+    final String propertyPath;
+    final String parentPath;
+    final String name;
+    /**
+     * Stores the parent path element in reverse order
+     * parentPath -> foo/bar/baz -> [baz, bar, foo]
+     */
+    final String[] ancestors;
+
+    public static boolean isRelativeProperty(String propertyName){
+        return !isAbsolute(propertyName) && PathUtils.getNextSlash(propertyName, 0) > 0;
+    }
+
+    public RelativeProperty(String propertyPath){
+        this.propertyPath = propertyPath;
+        name = PathUtils.getName(propertyPath);
+        parentPath = PathUtils.getParentPath(propertyPath);
+        ancestors = computeAncestors(parentPath);
+    }
+
+    @Nonnull
+    public NodeBuilder getPropDefnNode(NodeBuilder propNode) {
+        NodeBuilder result = propNode;
+        for (String name : elements(propertyPath)){
+            result = result.getChildNode(name);
+        }
+        return result;
+    }
+
+    @CheckForNull
+    public PropertyState getProperty(NodeState state) {
+        NodeState node = state;
+        for (String name : elements(parentPath)){
+            node = node.getChildNode(name);
+            if (!node.exists()){
+                return null;
+            }
+        }
+        return node.exists() ? node.getProperty(name) : null;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        RelativeProperty that = (RelativeProperty) o;
+
+        if (!propertyPath.equals(that.propertyPath)) return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        return propertyPath.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return propertyPath;
+    }
+
+    private String[] computeAncestors(String parentPath) {
+        return toArray(copyOf(elements(parentPath)).reverse(), String.class);
+    }
+}

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

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinitionTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinitionTest.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinitionTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexDefinitionTest.java Thu Oct 30 17:54:42 2014
@@ -21,6 +21,7 @@ package org.apache.jackrabbit.oak.plugin
 
 import javax.jcr.PropertyType;
 
+import com.google.common.collect.Iterables;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.apache.lucene.codecs.Codec;
@@ -108,5 +109,28 @@ public class IndexDefinitionTest {
         assertEquals(simple.getName(), defn.getCodec().getName());
     }
 
+    @Test
+    public void relativeProperty() throws Exception{
+        builder.setProperty(createProperty(INCLUDE_PROPERTY_NAMES, of("foo" , "foo1/bar"), STRINGS));
+        IndexDefinition defn = new IndexDefinition(builder);
+
+        assertEquals(1, defn.getRelativeProps().size());
+        assertEquals(new RelativeProperty("foo1/bar"), Iterables.getFirst(defn.getRelativeProps(), null));
+        assertTrue(defn.hasRelativeProperty("bar"));
+        assertFalse(defn.hasRelativeProperty("foo"));
+    }
 
+    @Test
+    public void relativePropertyConfig() throws Exception{
+        builder.child(PROP_NODE).child("foo1").child("bar").setProperty(LuceneIndexConstants.PROP_TYPE, PropertyType.TYPENAME_DATE);
+        builder.child(PROP_NODE).child("foo2").child("bar2").child("baz").setProperty(LuceneIndexConstants.PROP_TYPE, PropertyType.TYPENAME_LONG);
+        builder.setProperty(createProperty(INCLUDE_PROPERTY_NAMES, of("foo", "foo1/bar", "foo2/bar2/baz"), STRINGS));
+        IndexDefinition defn = new IndexDefinition(builder);
+
+        assertEquals(2, defn.getRelativeProps().size());
+        assertNull(defn.getPropDefn("foo"));
+        assertNotNull(defn.getPropDefn("foo1/bar"));
+        assertEquals(PropertyType.DATE, defn.getPropDefn("foo1/bar").getPropertyType());
+        assertEquals(PropertyType.LONG, defn.getPropDefn("foo2/bar2/baz").getPropertyType());
+    }
 }

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorTest.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditorTest.java Thu Oct 30 17:54:42 2014
@@ -153,6 +153,54 @@ public class LuceneIndexEditorTest {
         assertEquals(2, getSearcher().getIndexReader().numDocs());
     }
 
+    @Test
+    public void testLuceneWithRelativeProperty() throws Exception {
+        NodeBuilder index = builder.child(INDEX_DEFINITIONS_NAME);
+        NodeBuilder nb = newLuceneIndexDefinition(index, "lucene",
+                of(TYPENAME_STRING));
+        nb.setProperty(LuceneIndexConstants.FULL_TEXT_ENABLED, false);
+        nb.setProperty(createProperty(INCLUDE_PROPERTY_NAMES, of("foo", "jcr:content/mime",
+                "jcr:content/metadata/type"), STRINGS));
+
+        NodeState before = builder.getNodeState();
+        builder.child("test").setProperty("foo", "fox is jumping");
+        builder.child("test").child("jcr:content").setProperty("mime", "text");
+        builder.child("test").child("jcr:content").child("metadata").setProperty("type", "image");
+        builder.child("jcr:content").setProperty("count", "text");
+        builder.child("jcr:content").child("boom").child("metadata").setProperty("type", "image");
+        NodeState after = builder.getNodeState();
+
+        NodeState indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
+        tracker.update(indexed);
+
+        assertEquals(1, getSearcher().getIndexReader().numDocs());
+
+        assertEquals("/test", getPath(new TermQuery(new Term("foo", "fox is jumping"))));
+        assertEquals("/test", getPath(new TermQuery(new Term("jcr:content/mime", "text"))));
+        assertEquals("/test", getPath(new TermQuery(new Term("jcr:content/metadata/type", "image"))));
+        assertNull("bar must NOT be indexed", getPath(new TermQuery(new Term("count", "text"))));
+
+        releaseIndexNode();
+        before = indexed;
+        builder = before.builder();
+        builder.child("test").child("jcr:content").setProperty("mime", "pdf");
+        after = builder.getNodeState();
+        indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
+        tracker.update(indexed);
+
+        assertEquals("/test", getPath(new TermQuery(new Term("jcr:content/mime", "pdf"))));
+
+        releaseIndexNode();
+        before = indexed;
+        builder = before.builder();
+        builder.child("test").child("jcr:content").child("metadata").remove();
+        after = builder.getNodeState();
+        indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
+        tracker.update(indexed);
+        assertNull("relative removes must be persisted too",
+                getPath(new TermQuery(new Term("jcr:content/metadata/type", "image"))));
+    }
+
     //@Test
     public void checkLuceneIndexFileUpdates() throws Exception{
         NodeBuilder index = builder.child(INDEX_DEFINITIONS_NAME);

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndexTest.java?rev=1635563&r1=1635562&r2=1635563&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LucenePropertyIndexTest.java Thu Oct 30 17:54:42 2014
@@ -378,6 +378,20 @@ public class LucenePropertyIndexTest ext
     }
 
     @Test
+    public void testWithRelativeProperty() throws Exception{
+        Tree parent = root.getTree("/");
+        Tree idx = createIndex(parent, "test1", of("b/propa", "propb"));
+        root.commit();
+
+        Tree test = parent.addChild("test2");
+        test.addChild("a").addChild("b").setProperty("propa", "a");
+        root.commit();
+
+        assertQuery("select [jcr:path] from [nt:base] as s where [b/propa] = 'a'", asList("/test2/a"));
+
+    }
+
+    @Test
     public void indexDefinitionBelowRoot() throws Exception {
         Tree parent = root.getTree("/").addChild("test");
         Tree idx = createIndex(parent, "test1", of("propa", "propb"));
@@ -429,6 +443,30 @@ public class LucenePropertyIndexTest ext
         assertOrderedQuery("select [jcr:path] from [nt:base]  order by [foo] DESC", getSortedPaths(tuples, OrderDirection.DESC));
     }
 
+    @Test
+    public void sortQueriesWithLong_NotIndexed_relativeProps() throws Exception {
+        Tree idx = createIndex("test1", Collections.<String>emptySet());
+        idx.setProperty(createProperty(ORDERED_PROP_NAMES, of("foo/bar"), STRINGS));
+        Tree propIdx = idx.addChild(PROP_NODE).addChild("foo").addChild("bar");
+        propIdx.setProperty(LuceneIndexConstants.PROP_TYPE, PropertyType.TYPENAME_LONG);
+        root.commit();
+
+        assertThat(explain("select [jcr:path] from [nt:base] order by [foo/bar]"), containsString("lucene:test1"));
+
+        Tree test = root.getTree("/").addChild("test");
+        List<Long> values = createLongs(NUMBER_OF_NODES);
+        List<Tuple> tuples = Lists.newArrayListWithCapacity(values.size());
+        for(int i = 0; i < values.size(); i++){
+            Tree child = test.addChild("n"+i);
+            child.addChild("foo").setProperty("bar", values.get(i));
+            tuples.add(new Tuple(values.get(i), child.getPath()));
+        }
+        root.commit();
+
+        assertOrderedQuery("select [jcr:path] from [nt:base] order by [foo/bar]", getSortedPaths(tuples, OrderDirection.ASC));
+        assertOrderedQuery("select [jcr:path] from [nt:base]  order by [foo/bar] DESC", getSortedPaths(tuples, OrderDirection.DESC));
+    }
+
     void assertSortedLong() throws CommitFailedException {
         List<Tuple> tuples = createDataForLongProp();
         assertOrderedQuery("select [jcr:path] from [nt:base] where [bar] = 'baz' order by [foo]", getSortedPaths(tuples, OrderDirection.ASC));

Added: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativePropertyTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativePropertyTest.java?rev=1635563&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativePropertyTest.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/RelativePropertyTest.java Thu Oct 30 17:54:42 2014
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.junit.Test;
+
+import static org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent.INITIAL_CONTENT;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class RelativePropertyTest {
+    private NodeState root = INITIAL_CONTENT;
+
+    private NodeBuilder builder = root.builder();
+
+    @Test
+    public void relativeProperty() throws Exception{
+        assertTrue(RelativeProperty.isRelativeProperty("foo/bar"));
+        assertTrue(RelativeProperty.isRelativeProperty("foo/bar/baz"));
+        assertFalse(RelativeProperty.isRelativeProperty("/foo/bar/baz"));
+        assertFalse(RelativeProperty.isRelativeProperty("/"));
+        assertFalse(RelativeProperty.isRelativeProperty(""));
+    }
+
+    @Test
+    public void ancesstors() throws Exception{
+        RelativeProperty rp = new RelativeProperty("foo/bar/baz/boom");
+        assertArrayEquals(new String[] {"baz", "bar", "foo"}, rp.ancestors);
+    }
+
+    @Test
+    public void getPropDefnNode() throws Exception{
+        RelativeProperty rp = new RelativeProperty("foo/bar/baz");
+        builder.child("foo").child("bar").child("baz").setProperty("a", "b");
+        NodeBuilder propDefn = rp.getPropDefnNode(builder);
+        assertTrue(propDefn.exists());
+        assertEquals("b", propDefn.getString("a"));
+    }
+
+    @Test
+    public void testProperty() throws Exception{
+        RelativeProperty rp = new RelativeProperty("foo/bar/baz");
+        builder.child("foo").child("bar").setProperty("baz", "b");
+        PropertyState p = rp.getProperty(builder.getNodeState());
+        assertNotNull(p);
+        assertEquals("b", p.getValue(Type.STRING));
+
+        RelativeProperty rp2 = new RelativeProperty("a/b");
+        assertNull(rp2.getProperty(builder.getNodeState()));
+    }
+}

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