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 th...@apache.org on 2016/09/16 12:00:40 UTC

svn commit: r1761025 - in /jackrabbit/oak/trunk/oak-lucene/src: main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/ test/java/org/apache/jackrabbit/oak/jcr/ test/java/org/apache/jackr...

Author: thomasm
Date: Fri Sep 16 12:00:39 2016
New Revision: 1761025

URL: http://svn.apache.org/viewvc?rev=1761025&view=rev
Log:
OAK-3574 Query engine: support p=lowercase('x') and other function-based indexes

Added:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessor.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/FunctionIndexTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessorTest.java
Modified:
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/FieldNames.java
    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/IndexPlanner.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexEditor.java
    jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/PropertyDefinition.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
    jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/TestUtil.java

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/FieldNames.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/FieldNames.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/FieldNames.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/FieldNames.java Fri Sep 16 12:00:39 2016
@@ -90,6 +90,11 @@ public final class FieldNames {
      * Name of the field that contains the node name
      */
     public static final String NODE_NAME = ":nodeName";
+    
+    /**
+     * Suffix of the fields that contains function values
+     */
+    public static final String FUNCTION_PREFIX = "function*";
 
     /**
      * Used to select only the PATH field from the lucene documents

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=1761025&r1=1761024&r2=1761025&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 Fri Sep 16 12:00:39 2016
@@ -50,6 +50,7 @@ import org.apache.jackrabbit.oak.namepat
 import org.apache.jackrabbit.oak.plugins.index.IndexConstants;
 import org.apache.jackrabbit.oak.plugins.index.PathFilter;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.ConfigUtil;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.FunctionIndexProcessor;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.TokenizerChain;
 import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
 import org.apache.jackrabbit.oak.plugins.nodetype.ReadOnlyNodeTypeManager;
@@ -730,6 +731,7 @@ public final class IndexDefinition imple
         private final Map<String, PropertyDefinition> propConfigs;
         private final List<NamePattern> namePatterns;
         private final List<PropertyDefinition> nullCheckEnabledProperties;
+        private final List<PropertyDefinition> functionRestrictions;
         private final List<PropertyDefinition> notNullCheckEnabledProperties;
         private final List<PropertyDefinition> nodeScopeAnalyzedProps;
         private final boolean indexesAllNodesOfMatchingType;
@@ -755,17 +757,19 @@ public final class IndexDefinition imple
 
             List<NamePattern> namePatterns = newArrayList();
             List<PropertyDefinition> nonExistentProperties = newArrayList();
+            List<PropertyDefinition> functionRestrictions = newArrayList();
             List<PropertyDefinition> existentProperties = newArrayList();
             List<PropertyDefinition> nodeScopeAnalyzedProps = newArrayList();
             List<Aggregate.Include> propIncludes = newArrayList();
             this.propConfigs = collectPropConfigs(config, namePatterns, propIncludes, nonExistentProperties,
-                    existentProperties, nodeScopeAnalyzedProps);
+                    existentProperties, nodeScopeAnalyzedProps, functionRestrictions);
             this.propAggregate = new Aggregate(nodeTypeName, propIncludes);
             this.aggregate = combine(propAggregate, nodeTypeName);
 
             this.namePatterns = ImmutableList.copyOf(namePatterns);
             this.nodeScopeAnalyzedProps = ImmutableList.copyOf(nodeScopeAnalyzedProps);
             this.nullCheckEnabledProperties = ImmutableList.copyOf(nonExistentProperties);
+            this.functionRestrictions = ImmutableList.copyOf(functionRestrictions);
             this.notNullCheckEnabledProperties = ImmutableList.copyOf(existentProperties);
             this.fulltextEnabled = aggregate.hasNodeAggregates() || hasAnyFullTextEnabledProperty();
             this.nodeFullTextIndexed = aggregate.hasNodeAggregates() || anyNodeScopeIndexedProperty();
@@ -794,6 +798,7 @@ public final class IndexDefinition imple
             this.propAggregate = original.propAggregate;
             this.nullCheckEnabledProperties = original.nullCheckEnabledProperties;
             this.notNullCheckEnabledProperties = original.notNullCheckEnabledProperties;
+            this.functionRestrictions = original.functionRestrictions;
             this.nodeScopeAnalyzedProps = original.nodeScopeAnalyzedProps;
             this.aggregate = combine(propAggregate, nodeTypeName);
             this.fulltextEnabled = aggregate.hasNodeAggregates() || original.fulltextEnabled;
@@ -834,6 +839,10 @@ public final class IndexDefinition imple
         public List<PropertyDefinition> getNullCheckEnabledProperties() {
             return nullCheckEnabledProperties;
         }
+        
+        public List<PropertyDefinition> getFunctionRestrictions() {
+            return functionRestrictions;
+        }
 
         public List<PropertyDefinition> getNotNullCheckEnabledProperties() {
             return notNullCheckEnabledProperties;
@@ -957,11 +966,13 @@ public final class IndexDefinition imple
             return JcrConstants.NT_BASE.equals(baseNodeType);
         }
 
-        private Map<String, PropertyDefinition> collectPropConfigs(NodeState config, List<NamePattern> patterns,
+        private Map<String, PropertyDefinition> collectPropConfigs(NodeState config, 
+                                                                   List<NamePattern> patterns,
                                                                    List<Aggregate.Include> propAggregate,
                                                                    List<PropertyDefinition> nonExistentProperties,
                                                                    List<PropertyDefinition> existentProperties,
-                                                                   List<PropertyDefinition> nodeScopeAnalyzedProps) {
+                                                                   List<PropertyDefinition> nodeScopeAnalyzedProps, 
+                                                                   List<PropertyDefinition> functionRestrictions) {
             Map<String, PropertyDefinition> propDefns = newHashMap();
             NodeState propNode = config.getChildNode(LuceneIndexConstants.PROP_NODE);
 
@@ -981,6 +992,18 @@ public final class IndexDefinition imple
                 NodeState propDefnNode = propNode.getChildNode(propName);
                 if (propDefnNode.exists() && !propDefns.containsKey(propName)) {
                     PropertyDefinition pd = new PropertyDefinition(this, propName, propDefnNode);
+                    if (pd.function != null) {
+                        functionRestrictions.add(pd);
+                        String[] properties = FunctionIndexProcessor.getProperties(pd.functionCode);
+                        for (String p : properties) {
+                            if (PathUtils.getDepth(p) > 1) {
+                                PropertyDefinition pd2 = new PropertyDefinition(this, p, propDefnNode);
+                                propAggregate.add(new Aggregate.PropertyInclude(pd2));
+                            }
+                        }
+                        // a function index has no other options
+                        continue;
+                    }
                     if(pd.isRegexp){
                         patterns.add(new NamePattern(pd.name, pd));
                     } else {

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexPlanner.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexPlanner.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexPlanner.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexPlanner.java Fri Sep 16 12:00:39 2016
@@ -146,6 +146,15 @@ class IndexPlanner {
 
         List<String> indexedProps = newArrayListWithCapacity(filter.getPropertyRestrictions().size());
 
+        for (PropertyDefinition functionIndex : indexingRule.getFunctionRestrictions()) {
+            for (PropertyRestriction pr : filter.getPropertyRestrictions()) {
+                String f = functionIndex.function;
+                if (pr.propertyName.equals(f)) {
+                    indexedProps.add(f);
+                    result.propDefns.put(f, functionIndex);
+                }
+            }
+        }
         //Optimization - Go further only if any of the property is configured
         //for property index
         List<String> facetFields = new LinkedList<String>();
@@ -156,7 +165,7 @@ class IndexPlanner {
                     continue;
                 }
                 if (name.startsWith(QueryConstants.FUNCTION_RESTRICTION_PREFIX)) {
-                    // TODO support function-based indexes
+                    // function-based indexes were handled before
                     continue;
                 }
                 if (QueryImpl.REP_FACET.equals(pr.propertyName)) {
@@ -504,6 +513,13 @@ class IndexPlanner {
                 // Supports jcr:score descending natively
                 orderEntries.add(IndexDefinition.NATIVE_SORT_ORDER);
             }
+            for (PropertyDefinition functionIndex : rule.getFunctionRestrictions()) {
+                if (o.getPropertyName().equals(functionIndex.function)) {
+                    // Lucene can manage any order desc/asc
+                    orderEntries.add(o);
+                    result.sortedProperties.add(functionIndex);
+                }
+            }
         }
 
         //TODO Should we return order entries only when all order clauses are satisfied
@@ -549,6 +565,14 @@ class IndexPlanner {
             //Relative parent properties where [../foo1] is not null
             return true;
         }
+        boolean failTestOnMissingFunctionIndex = true;
+        if (failTestOnMissingFunctionIndex) {
+            // this means even just function restrictions fail the test
+            // (for example "where upper(name) = 'X'", 
+            // if a matching function-based index is missing
+            return false;
+        }
+        // the following would ensure the test doesn't fail in that case:
         for (PropertyRestriction r : filter.getPropertyRestrictions()) {
             if (!r.propertyName.startsWith(QueryConstants.FUNCTION_RESTRICTION_PREFIX)) {
                 // not a function restriction

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexConstants.java Fri Sep 16 12:00:39 2016
@@ -349,4 +349,9 @@ public interface LuceneIndexConstants {
      * Boolean property indicate that property should not be included in aggregation
      */
     String PROP_EXCLUDE_FROM_AGGREGATE = "excludeFromAggregation";
+
+    /**
+     * String property: the function to index, for function-based index
+     */
+    String PROP_FUNCTION = "function";
 }

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=1761025&r1=1761024&r2=1761025&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 Fri Sep 16 12:00:39 2016
@@ -18,6 +18,7 @@ package org.apache.jackrabbit.oak.plugin
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -31,7 +32,6 @@ import org.apache.jackrabbit.JcrConstant
 import org.apache.jackrabbit.oak.api.Blob;
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.api.PropertyState;
-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.commons.io.LazyInputStream;
@@ -40,10 +40,10 @@ import org.apache.jackrabbit.oak.plugins
 import org.apache.jackrabbit.oak.plugins.index.fulltext.ExtractedText;
 import org.apache.jackrabbit.oak.plugins.index.fulltext.ExtractedText.ExtractionResult;
 import org.apache.jackrabbit.oak.plugins.index.lucene.Aggregate.Matcher;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.FunctionIndexProcessor;
 import org.apache.jackrabbit.oak.plugins.index.lucene.writer.LuceneIndexWriter;
 import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
 import org.apache.jackrabbit.oak.plugins.memory.StringPropertyState;
-import org.apache.jackrabbit.oak.plugins.tree.TreeFactory;
 import org.apache.jackrabbit.oak.spi.commit.Editor;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.apache.jackrabbit.oak.util.BlobByteSource;
@@ -325,6 +325,7 @@ public class LuceneIndexEditor implement
 
         dirty |= indexAggregates(path, fields, state);
         dirty |= indexNullCheckEnabledProps(path, fields, state);
+        dirty |= indexFunctionRestrictions(path, fields, state);
         dirty |= indexNotNullCheckEnabledProps(path, fields, state);
 
         dirty |= augmentCustomFields(path, fields, state);
@@ -660,6 +661,31 @@ public class LuceneIndexEditor implement
         return fieldAdded;
     }
 
+    private boolean indexFunctionRestrictions(String path, List<Field> fields, NodeState state) {
+        boolean fieldAdded = false;
+        for (PropertyDefinition pd : indexingRule.getFunctionRestrictions()) {
+            PropertyState functionValue = calculateValue(path, state, pd.functionCode);
+            if (functionValue != null) {
+                if (pd.ordered) {
+                    addTypedOrderedFields(fields, functionValue, pd.function, pd);
+                }
+                addTypedFields(fields, functionValue, pd.function);
+                fieldAdded = true;
+            }
+        }
+        return fieldAdded;
+    }
+    
+    private static PropertyState calculateValue(String path, NodeState state, String[] functionCode) {
+        try {
+            return FunctionIndexProcessor.tryCalculateValue(path, state, functionCode);
+        } catch (RuntimeException e) {
+            log.error("Failed to calculate function value for {} at {}", 
+                    Arrays.toString(functionCode), path, e);
+            throw e;
+        }
+    }
+
     private boolean indexIfSinglePropertyRemoved() {
         boolean dirty = false;
         for (PropertyState ps : propertiesModified) {

Modified: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/PropertyDefinition.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/PropertyDefinition.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/PropertyDefinition.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/PropertyDefinition.java Fri Sep 16 12:00:39 2016
@@ -26,6 +26,7 @@ import org.apache.jackrabbit.oak.api.Pro
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.plugins.index.lucene.IndexDefinition.IndexingRule;
+import org.apache.jackrabbit.oak.plugins.index.lucene.util.FunctionIndexProcessor;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.slf4j.Logger;
@@ -99,6 +100,16 @@ class PropertyDefinition {
     @CheckForNull
     final String nonRelativeName;
 
+    /**
+     * For function-based indexes: the function name, in Polish notation.
+     */    
+    final String function;
+    
+    /**
+     * For function-based indexes: the function code, as tokens.
+     */    
+    final String[] functionCode;
+
     public PropertyDefinition(IndexingRule idxDefn, String nodeName, NodeState defn) {
         this.isRegexp = getOptionalValue(defn, PROP_IS_REGEX, false);
         this.name = getName(defn, nodeName);
@@ -134,6 +145,9 @@ class PropertyDefinition {
         this.nonRelativeName = determineNonRelativeName();
         this.ancestors = computeAncestors(name);
         this.facet = getOptionalValueIfIndexed(defn, LuceneIndexConstants.PROP_FACETS, false);
+        this.function = FunctionIndexProcessor.convertToPolishNotation(
+                getOptionalValue(defn, LuceneIndexConstants.PROP_FUNCTION, null));
+        this.functionCode = FunctionIndexProcessor.getFunctionCode(this.function);
         validate();
     }
 

Added: jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessor.java?rev=1761025&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessor.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessor.java Fri Sep 16 12:00:39 2016
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.jackrabbit.oak.plugins.index.lucene.util;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
+import org.apache.jackrabbit.oak.spi.query.QueryConstants;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A parser for function-based indexes. It converts the human-readable function
+ * definition (XPath) to the internal Polish notation.
+ */
+public class FunctionIndexProcessor {
+    
+    private static final Logger LOG =
+            LoggerFactory.getLogger(FunctionIndexProcessor.class);
+    
+    private String remaining;
+    
+    private FunctionIndexProcessor(String function) {
+        this.remaining = function;
+    }
+    
+    /**
+     * Get the list of properties used in the given function code.
+     * 
+     * @param functionCode the tokens, for example ["function", "lower", "@name"]
+     * @return the list of properties, for example ["name"]
+     */
+    public static String[] getProperties(String[] functionCode) {
+        ArrayList<String> properties = new ArrayList<String>();
+        for(String token : functionCode) {
+            if (token.startsWith("@")) {
+                String propertyName = token.substring(1);
+                properties.add(propertyName);
+            }
+        }
+        return properties.toArray(new String[0]);
+    }
+
+    /**
+     * Try to calculate the value for the given function code.
+     * 
+     * @param path the path of the node
+     * @param state the node state
+     * @param functionCode the tokens, for example ["function", "lower", "@name"]
+     * @return null, or the calculated value
+     */
+    public static PropertyState tryCalculateValue(String path, NodeState state, String[] functionCode) {
+        Deque<PropertyState> stack = new ArrayDeque<PropertyState>();
+        for (int i = functionCode.length - 1; i > 0; i--) {
+            String token = functionCode[i];
+            PropertyState ps;
+            if (token.startsWith("@")) {
+                String propertyName = token.substring(1);
+                ps = getProperty(path, state, propertyName);
+            } else {
+                ps = calculateFunction(token, stack);
+            }
+            if (ps == null) {
+                // currently, all operations involving null return null
+                return null;
+            }
+            stack.push(ps);
+        }
+        return stack.pop();
+    }
+    
+    /**
+     * Split the polish notation into a tokens that can more easily be processed.
+     *  
+     *  @param functionDescription in polish notation, for example "function*lower*{@literal @}name"
+     *  @return tokens, for example ["function", "lower", "{@literal @}name"]
+     */
+    public static String[] getFunctionCode(String functionDescription) {
+        if (functionDescription == null) {
+            return null;
+        }
+        return functionDescription.split("\\*");
+    }
+    
+    private static PropertyState calculateFunction(String functionName, 
+            Deque<PropertyState> stack) {
+        PropertyState ps = stack.pop();
+        Type<?> type = null;
+        ArrayList<Object> values = new ArrayList<Object>(ps.count());
+        for (int i = 0; i < ps.count(); i++) {
+            String s = ps.getValue(Type.STRING, i);
+            Object x;
+            if ("lower".equals(functionName)) {
+                x = s.toLowerCase();
+                type = Type.STRING;
+            } else if ("upper".equals(functionName)) {
+                x = s.toUpperCase();
+                type = Type.STRING;
+            } else if ("length".equals(functionName)) {
+                x = (long) s.length();
+                type = Type.LONG;
+            } else {
+                LOG.debug("Unknown function {}", functionName);
+                return null;
+            }
+            values.add(x);
+        }
+        PropertyState result;
+        if (values.size() == 1) {
+            result = PropertyStates.createProperty("value", values.get(0), type);
+        } else {
+            type = type.getArrayType();
+            result = PropertyStates.createProperty("value", values, type);
+        }
+        return result;
+    }
+    
+    private static PropertyState getProperty(String path, NodeState state, 
+            String propertyName) {
+        if (PathUtils.getDepth(propertyName) != 1) {
+            for(String n : PathUtils.elements(PathUtils.getParentPath(propertyName))) {
+                state = state.getChildNode(n);
+                if (!state.exists()) {
+                    return null;
+                }
+            }
+            propertyName = PathUtils.getName(propertyName);
+        }
+        PropertyState ps;
+        if (":localname".equals(propertyName)) {
+            ps = PropertyStates.createProperty("value", 
+                    getLocalName(PathUtils.getName(path)), Type.STRING);
+        } else if (":name".equals(propertyName)) {
+            ps = PropertyStates.createProperty("value", 
+                    PathUtils.getName(path), Type.STRING);
+        } else {
+            ps = state.getProperty(propertyName);
+        }
+        if (ps == null || ps.count() == 0) {
+            return null;
+        }
+        return ps;
+    }
+    
+    private static String getLocalName(String name) {
+        int colon = name.indexOf(':');
+        // TODO LOCALNAME: evaluation of local name might not be correct
+        return colon < 0 ? name : name.substring(colon + 1);
+    }
+    
+    /**
+     * Convert a function (in human-readable form) to the polish notation.
+     * 
+     * @param function the function, for example "lower([name])"
+     * @return the polish notation, for example "function*lower*{@literal @}name"
+     */
+    public static String convertToPolishNotation(String function) {
+        if (function == null) {
+            return null;
+        }
+        FunctionIndexProcessor p = new FunctionIndexProcessor(function);
+        return QueryConstants.FUNCTION_RESTRICTION_PREFIX + p.parse();
+    }
+    
+    String parse() {
+        if (match("fn:local-name()") || match("localname()")) {
+            return "@:localname";
+        }
+        if (match("fn:name()") || match("name()")) {
+            return "@:name";
+        }
+        if (match("fn:upper-case(") || match("upper(")) {
+            return "upper*" + parse() + read(")");
+        }
+        if (match("fn:lower-case(") || match("lower(")) {
+            return "lower*" + parse() + read(")");
+        }
+        if (match("fn:string-length(") || match("length(")) {
+            return "length*" + parse() + read(")");
+        }
+        int end = remaining.indexOf(')');
+        if (end >= 0) {
+            remaining = remaining.substring(0, end);
+        }
+        if (remaining.startsWith("[")) {
+            return property(remaining.substring(1, remaining.lastIndexOf(']')).replaceAll("]]", "]"));
+        }
+        // property name
+        return property(remaining.replaceAll("@", ""));
+    }
+    
+    String property(String p) {
+        return "@" + p;
+    }
+    
+    private String read(String string) {
+        match(string);
+        return "";
+    }
+    
+    private boolean match(String string) {
+        if (remaining.startsWith(string)) {
+            remaining = remaining.substring(string.length());
+            return true;
+        }
+        return false;
+    }
+
+}

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java Fri Sep 16 12:00:39 2016
@@ -100,6 +100,14 @@ public class LuceneOakRepositoryStub ext
                 NodeBuilder props = ntBase.child(LuceneIndexConstants.PROP_NODE);
                 props.setProperty(JCR_PRIMARYTYPE, "nt:unstructured", NAME);
 
+                // Enable function-based indexes: upper+lower(name+localname+prop1)
+                functionBasedIndex(props, "upper(name())");
+                functionBasedIndex(props, "lower(name())");
+                functionBasedIndex(props, "upper(localname())");
+                functionBasedIndex(props, "lower(localname())");
+                functionBasedIndex(props, "upper([prop1])");
+                functionBasedIndex(props, "lower([prop1])");
+
                 enableFulltextIndex(props.child("allProps"));
             }
         }
@@ -115,5 +123,12 @@ public class LuceneOakRepositoryStub ext
                     .setProperty(LuceneIndexConstants.PROP_NAME, LuceneIndexConstants.REGEX_ALL_PROPS)
                     .setProperty(LuceneIndexConstants.PROP_IS_REGEX, true);
         }
+        
+        private static void functionBasedIndex(NodeBuilder props, String function) {
+            props.child(function).
+                setProperty(JCR_PRIMARYTYPE, "nt:unstructured", NAME).
+                setProperty(LuceneIndexConstants.PROP_FUNCTION, function);
+        }
+
     }
 }

Added: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/FunctionIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/FunctionIndexTest.java?rev=1761025&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/FunctionIndexTest.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/FunctionIndexTest.java Fri Sep 16 12:00:39 2016
@@ -0,0 +1,314 @@
+/*
+ * 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 static org.apache.jackrabbit.oak.api.QueryEngine.NO_BINDINGS;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NODE_TYPE;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.REINDEX_PROPERTY_NAME;
+import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
+import java.text.ParseException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import javax.jcr.PropertyType;
+
+import org.apache.jackrabbit.JcrConstants;
+import org.apache.jackrabbit.oak.Oak;
+import org.apache.jackrabbit.oak.api.ContentRepository;
+import org.apache.jackrabbit.oak.api.Result;
+import org.apache.jackrabbit.oak.api.ResultRow;
+import org.apache.jackrabbit.oak.api.Tree;
+import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.nodetype.NodeTypeIndexProvider;
+import org.apache.jackrabbit.oak.plugins.index.property.PropertyIndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryNodeStore;
+import org.apache.jackrabbit.oak.plugins.memory.PropertyStates;
+import org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent;
+import org.apache.jackrabbit.oak.query.AbstractQueryTest;
+import org.apache.jackrabbit.oak.spi.commit.Observer;
+import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
+import org.apache.jackrabbit.oak.spi.security.OpenSecurityProvider;
+import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+public class FunctionIndexTest extends AbstractQueryTest {
+
+    private LuceneIndexEditorProvider editorProvider;
+
+    private NodeStore nodeStore;
+    
+    @Override
+    protected ContentRepository createRepository() {
+        editorProvider = new LuceneIndexEditorProvider();
+        LuceneIndexProvider provider = new LuceneIndexProvider();
+        nodeStore = new MemoryNodeStore();
+        return new Oak(nodeStore)
+                .with(new InitialContent())
+                .with(new OpenSecurityProvider())
+                .with((QueryIndexProvider) provider)
+                .with((Observer) provider)
+                .with(editorProvider)
+                .with(new PropertyIndexEditorProvider())
+                .with(new NodeTypeIndexProvider())
+                .createContentRepository();
+    }
+    
+    @Test
+    public void noIndexTest() throws Exception {
+        Tree test = root.getTree("/").addChild("test");
+        for (int idx = 0; idx < 3; idx++) {
+            Tree low = test.addChild("" + (char) ('a' + idx));
+            low.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            Tree up = test.addChild("" + (char) ('A' + idx));
+            up.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+        }
+        root.commit();
+        
+        String query = "select [jcr:path] from [nt:base] where lower(localname()) = 'b'";
+        assertThat(explain(query), containsString("traverse"));
+        assertQuery(query, Lists.newArrayList("/test/b", "/test/B"));
+        
+        String queryXPath = "/jcr:root/test//*[fn:lower-case(fn:local-name()) = 'b']";
+        assertThat(explainXpath(queryXPath), containsString("traverse"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/b", "/test/B"));
+
+        queryXPath = "/jcr:root/test//*[fn:lower-case(fn:local-name()) > 'b']";
+        assertThat(explainXpath(queryXPath), containsString("traverse"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/c", "/test/C"));
+
+        query = "select [jcr:path] from [nt:base] where lower(localname()) = 'B'";
+        assertThat(explain(query), containsString("traverse"));
+        assertQuery(query, Lists.<String>newArrayList());
+    }
+    
+    @Test
+    public void lowerCaseLocalName() throws Exception {
+        Tree luceneIndex = createIndex("lowerLocalName", Collections.<String>emptySet());
+        luceneIndex.setProperty("excludedPaths", 
+                Lists.newArrayList("/jcr:system", "/oak:index"), Type.STRINGS);
+        Tree func = luceneIndex.addChild(LuceneIndexConstants.INDEX_RULES)
+                .addChild("nt:base")
+                .addChild(LuceneIndexConstants.PROP_NODE)
+                .addChild("lowerLocalName");
+        func.setProperty(LuceneIndexConstants.PROP_FUNCTION, "lower(localname())");
+
+        Tree test = root.getTree("/").addChild("test");
+        for (int idx = 0; idx < 3; idx++) {
+            Tree low = test.addChild("" + (char) ('a' + idx));
+            low.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            Tree up = test.addChild("" + (char) ('A' + idx));
+            up.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+        }
+        root.commit();
+
+        String query = "select [jcr:path] from [nt:base] where lower(localname()) = 'b'";
+        assertThat(explain(query), containsString("lucene:lowerLocalName"));
+        assertQuery(query, Lists.newArrayList("/test/b", "/test/B"));
+        
+        String queryXPath = "/jcr:root//*[fn:lower-case(fn:local-name()) = 'b']";
+        assertThat(explainXpath(queryXPath), containsString("lucene:lowerLocalName"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/b", "/test/B"));
+
+        queryXPath = "/jcr:root//*[fn:lower-case(fn:local-name()) > 'b']";
+        assertThat(explainXpath(queryXPath), containsString("lucene:lowerLocalName"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/c", "/test/C", "/test"));
+
+        query = "select [jcr:path] from [nt:base] where lower(localname()) = 'B'";
+        assertThat(explain(query), containsString("lucene:lowerLocalName"));
+        assertQuery(query, Lists.<String>newArrayList());
+    }
+    
+    @Test
+    public void lengthName() throws Exception {
+        Tree luceneIndex = createIndex("lengthName", Collections.<String>emptySet());
+        luceneIndex.setProperty("excludedPaths", 
+                Lists.newArrayList("/jcr:system", "/oak:index"), Type.STRINGS);
+        Tree func = luceneIndex.addChild(LuceneIndexConstants.INDEX_RULES)
+                .addChild("nt:base")
+                .addChild(LuceneIndexConstants.PROP_NODE)
+                .addChild("lengthName");
+        func.setProperty(LuceneIndexConstants.PROP_ORDERED, true);
+        func.setProperty(LuceneIndexConstants.PROP_TYPE, PropertyType.TYPENAME_LONG);
+        func.setProperty(LuceneIndexConstants.PROP_FUNCTION, "fn:string-length(fn:name())");
+
+        Tree test = root.getTree("/").addChild("test");
+        for (int idx = 1; idx < 1000; idx *= 10) {
+            Tree testNode = test.addChild("test" + idx);
+            testNode.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+        }
+        root.commit();
+
+        String query = "select [jcr:path] from [nt:base] where length(name()) = 6";
+        assertThat(explain(query), containsString("lucene:lengthName"));
+        assertQuery(query, Lists.newArrayList("/test/test10"));
+        
+        String queryXPath = "/jcr:root//*[fn:string-length(fn:name()) = 7]";
+        assertThat(explainXpath(queryXPath), containsString("lucene:lengthName"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/test100"));
+
+        queryXPath = "/jcr:root//* order by fn:string-length(fn:name())";
+        assertThat(explainXpath(queryXPath), containsString("lucene:lengthName"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList(
+                "/test", "/test/test1", "/test/test10", "/test/test100"));
+    }
+    
+    @Test
+    public void length() throws Exception {
+        Tree luceneIndex = createIndex("length", Collections.<String>emptySet());
+        luceneIndex.setProperty("excludedPaths", 
+                Lists.newArrayList("/jcr:system", "/oak:index"), Type.STRINGS);
+        Tree func = luceneIndex.addChild(LuceneIndexConstants.INDEX_RULES)
+                .addChild("nt:base")
+                .addChild(LuceneIndexConstants.PROP_NODE)
+                .addChild("lengthName");
+        func.setProperty(LuceneIndexConstants.PROP_FUNCTION, "fn:string-length(@value)");
+
+        Tree test = root.getTree("/").addChild("test");
+        for (int idx = 1; idx <= 1000; idx *= 10) {
+            Tree testNode = test.addChild("test" + idx);
+            testNode.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            testNode.setProperty("value", new byte[idx]);
+        }
+        root.commit();
+
+        String query = "select [jcr:path] from [nt:base] where length([value]) = 100";
+        assertThat(explain(query), containsString("lucene:length"));
+        assertQuery(query, Lists.newArrayList("/test/test100"));
+        
+        String queryXPath = "/jcr:root//*[fn:string-length(@value) = 10]";
+        assertThat(explainXpath(queryXPath), containsString("lucene:length"));
+        assertQuery(queryXPath, "xpath", Lists.newArrayList("/test/test10"));
+    }
+    
+    @Test
+    public void upperCase() throws Exception {
+        Tree luceneIndex = createIndex("upper", Collections.<String>emptySet());
+        Tree func = luceneIndex.addChild(LuceneIndexConstants.INDEX_RULES)
+                .addChild("nt:base")
+                .addChild(LuceneIndexConstants.PROP_NODE)
+                .addChild("upperName");
+        func.setProperty(LuceneIndexConstants.PROP_FUNCTION, "fn:upper-case(@name)");
+
+        Tree test = root.getTree("/").addChild("test");
+        test.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+
+        List<String> paths = Lists.newArrayList();
+        for (int idx = 0; idx < 15; idx++) {
+            Tree a = test.addChild("n"+idx);
+            a.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            a.setProperty("name", "10% foo");
+            paths.add("/test/n" + idx);
+        }
+        root.commit();
+
+        String query = "select [jcr:path] from [nt:unstructured] where upper([name]) = '10% FOO'";
+        assertThat(explain(query), containsString("lucene:upper"));
+        assertQuery(query, paths);
+        
+        query = "select [jcr:path] from [nt:unstructured] where upper([name]) like '10\\% FOO'";
+        assertThat(explain(query), containsString("lucene:upper"));
+        assertQuery(query, paths);        
+    }
+
+    @Test
+    public void upperCaseRelative() throws Exception {
+        Tree luceneIndex = createIndex("upper", Collections.<String>emptySet());
+        Tree func = luceneIndex.addChild(LuceneIndexConstants.INDEX_RULES)
+                .addChild("nt:base")
+                .addChild(LuceneIndexConstants.PROP_NODE)
+                .addChild("upperName");
+        func.setProperty(LuceneIndexConstants.PROP_FUNCTION, "upper([data/name])");
+
+        Tree test = root.getTree("/").addChild("test");
+        test.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+
+        List<String> paths = Lists.newArrayList();
+        for (int idx = 0; idx < 15; idx++) {
+            Tree a = test.addChild("n"+idx);
+            a.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            Tree b = a.addChild("data");
+            b.setProperty("jcr:primaryType", "nt:unstructured", Type.NAME);
+            b.setProperty("name", "foo");
+            paths.add("/test/n" + idx);
+        }
+        root.commit();
+
+        String query = "select [jcr:path] from [nt:unstructured] where upper([data/name]) = 'FOO'";
+        assertThat(explain(query), containsString("lucene:upper"));
+        assertQuery(query, paths);
+        
+        String queryXPath = "/jcr:root//element(*, nt:unstructured)[fn:upper-case(data/@name) = 'FOO']";
+        assertThat(explainXpath(queryXPath), containsString("lucene:upper"));
+        assertQuery(queryXPath, "xpath", paths);
+        
+        for (int idx = 0; idx < 15; idx++) {
+            Tree a = test.getChild("n"+idx);
+            Tree b = a.getChild("data");
+            b.setProperty("name", "bar");
+        }
+        root.commit();
+        
+        query = "select [jcr:path] from [nt:unstructured] where upper([data/name]) = 'BAR'";
+        assertThat(explain(query), containsString("lucene:upper"));
+        assertQuery(query, paths);
+        
+        queryXPath = "/jcr:root//element(*, nt:unstructured)[fn:upper-case(data/@name) = 'BAR']";
+        assertThat(explainXpath(queryXPath), containsString("lucene:upper"));
+        assertQuery(queryXPath, "xpath", paths);
+    }
+    
+    protected String explain(String query){
+        String explain = "explain " + query;
+        return executeQuery(explain, "JCR-SQL2").get(0);
+    }
+
+    protected String explainXpath(String query) throws ParseException {
+        String explain = "explain " + query;
+        Result result = executeQuery(explain, "xpath", NO_BINDINGS);
+        ResultRow row = Iterables.getOnlyElement(result.getRows());
+        String plan = row.getValue("plan").getValue(Type.STRING);
+        return plan;
+    }
+    
+    protected Tree createIndex(String name, Set<String> propNames) {
+        Tree index = root.getTree("/");
+        return createIndex(index, name, propNames);
+    }
+    
+    static Tree createIndex(Tree index, String name, Set<String> propNames) {
+        Tree def = index.addChild(INDEX_DEFINITIONS_NAME).addChild(name);
+        def.setProperty(JcrConstants.JCR_PRIMARYTYPE,
+                INDEX_DEFINITIONS_NODE_TYPE, Type.NAME);
+        def.setProperty(TYPE_PROPERTY_NAME, LuceneIndexConstants.TYPE_LUCENE);
+        def.setProperty(REINDEX_PROPERTY_NAME, true);
+        def.setProperty(LuceneIndexConstants.FULL_TEXT_ENABLED, false);
+        def.setProperty(PropertyStates.createProperty(LuceneIndexConstants.INCLUDE_PROPERTY_NAMES, propNames, Type.STRINGS));
+        def.setProperty(LuceneIndexConstants.SAVE_DIR_LISTING, true);
+        return index.getChild(INDEX_DEFINITIONS_NAME).getChild(name);
+    }    
+
+}

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java Fri Sep 16 12:00:39 2016
@@ -65,6 +65,9 @@ public class LuceneIndexQueryTest extend
         TestUtil.enableForFullText(props, LuceneIndexConstants.REGEX_ALL_PROPS, true);
         TestUtil.enablePropertyIndex(props, "a/name", false);
         TestUtil.enablePropertyIndex(props, "b/name", false);
+        TestUtil.enableFunctionIndex(props, "length([name])");
+        TestUtil.enableFunctionIndex(props, "lower([name])");
+        TestUtil.enableFunctionIndex(props, "upper([name])");
         
         root.commit();                
     }

Modified: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/TestUtil.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/TestUtil.java?rev=1761025&r1=1761024&r2=1761025&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/TestUtil.java (original)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/TestUtil.java Fri Sep 16 12:00:39 2016
@@ -126,6 +126,12 @@ public class TestUtil {
         return prop;
     }
 
+    public static Tree enableFunctionIndex(Tree props, String function) {
+        Tree prop = props.addChild(unique("prop"));
+        prop.setProperty(LuceneIndexConstants.PROP_FUNCTION, function);
+        return prop;
+    }
+
     public static AggregatorBuilder newNodeAggregator(Tree indexDefn){
         return new AggregatorBuilder(indexDefn);
     }

Added: jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessorTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessorTest.java?rev=1761025&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessorTest.java (added)
+++ jackrabbit/oak/trunk/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/util/FunctionIndexProcessorTest.java Fri Sep 16 12:00:39 2016
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.jackrabbit.oak.plugins.index.lucene.util;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+
+public class FunctionIndexProcessorTest {
+    
+    @Test
+    public void getProperties() {
+        assertEquals(
+                "[a, test/b, test/:name]",
+                Arrays.toString(
+                FunctionIndexProcessor.getProperties(new String[] { "function",
+                        "multiply", "@a", "add", "@test/b", "@test/:name" })));
+    }
+    
+    @Test
+    public void xpath() {
+        checkConvert(
+                "fn:upper-case(@data)", 
+                "function*upper*@data");
+        checkConvert(
+                "fn:lower-case(test/@data)", 
+                "function*lower*@test/data");
+        checkConvert(
+                "fn:lower-case(fn:name())", 
+                "function*lower*@:name");        
+        checkConvert(
+                "fn:lower-case(fn:local-name())", 
+                "function*lower*@:localname");
+        checkConvert(
+                "fn:string-length(test/@data)", 
+                "function*length*@test/data");
+        checkConvert(
+                "fn:string-length(fn:name())", 
+                "function*length*@:name");
+        checkConvert(
+                "fn:lower-case(fn:upper-case(test/@data))", 
+                "function*lower*upper*@test/data");
+    }
+
+    @Test
+    public void sql2() {
+        checkConvert(
+                "upper([data])", 
+                "function*upper*@data");
+        checkConvert(
+                "lower([test/data])", 
+                "function*lower*@test/data");
+        checkConvert(
+                "lower(name())", 
+                "function*lower*@:name");
+        checkConvert(
+                "lower(localname())", 
+                "function*lower*@:localname");
+        checkConvert(
+                "length([test/data])", 
+                "function*length*@test/data");
+        checkConvert(
+                "length(name())", 
+                "function*length*@:name");
+        checkConvert(
+                "lower(upper([test/data]))", 
+                "function*lower*upper*@test/data");
+        // the ']' character is escaped as ']]'
+        checkConvert(
+                "[strange[0]]]", 
+                "function*@strange[0]");
+    }
+
+    private static void checkConvert(String function, String expectedPolishNotation) {
+        String p = FunctionIndexProcessor.convertToPolishNotation(function);
+        assertEquals(expectedPolishNotation, p);
+    }
+
+}