You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by al...@apache.org on 2014/04/16 15:46:53 UTC

svn commit: r1587906 - in /jackrabbit/oak/branches/1.0: oak-core/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/ oak-lucene/ oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ oak-lucene/src/test/java/org/apache/...

Author: alexparvulescu
Date: Wed Apr 16 13:46:53 2014
New Revision: 1587906

URL: http://svn.apache.org/r1587906
Log:
OAK-1722 Use a shared IndexSearcher for performing Lucene queries
OAK-925 Query: Lucene index loads all path in memory
 merged to branch 1.0


Added:
    jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java
      - copied unchanged from r1587428, jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexNode.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexTracker.java
      - copied unchanged from r1587428, jackrabbit/oak/trunk/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/IndexTracker.java
Modified:
    jackrabbit/oak/branches/1.0/oak-core/   (props changed)
    jackrabbit/oak/branches/1.0/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/MemoryNodeStore.java
    jackrabbit/oak/branches/1.0/oak-lucene/   (props changed)
    jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/query/MultiSessionQueryTest.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexAggregationTest.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexExclusionQueryTest.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
    jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java

Propchange: jackrabbit/oak/branches/1.0/oak-core/
------------------------------------------------------------------------------
--- svn:mergeinfo (added)
+++ svn:mergeinfo Wed Apr 16 13:46:53 2014
@@ -0,0 +1,2 @@
+/jackrabbit/oak/trunk/oak-core:1584578,1584602,1584614,1584616,1584709,1584781,1584937,1585297,1585304-1585305,1585420,1585424,1585427,1585448,1585465,1585468,1585486,1585497,1585509,1585647,1585655-1585656,1585661,1585665-1585666,1585669-1585670,1585673,1585680,1585719,1585763,1585770,1585896,1585904,1585907,1585940,1585949,1585951,1585956,1585962,1586287,1586320,1586364,1586372,1586655,1586836,1587130,1587224,1587399,1587408,1587428,1587472,1587485,1587538,1587580
+/jackrabbit/trunk/oak-core:1345480

Modified: jackrabbit/oak/branches/1.0/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/MemoryNodeStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/MemoryNodeStore.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/MemoryNodeStore.java (original)
+++ jackrabbit/oak/branches/1.0/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/MemoryNodeStore.java Wed Apr 16 13:46:53 2014
@@ -23,6 +23,7 @@ import static com.google.common.collect.
 import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
 import static org.apache.jackrabbit.oak.plugins.memory.ModifiedNodeState.squeeze;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Map;
@@ -38,6 +39,8 @@ import org.apache.jackrabbit.oak.api.Blo
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.spi.commit.CommitHook;
 import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.Observable;
+import org.apache.jackrabbit.oak.spi.commit.Observer;
 import org.apache.jackrabbit.oak.spi.state.ConflictAnnotatingRebaseDiff;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
@@ -48,12 +51,14 @@ import org.apache.jackrabbit.oak.spi.sta
  * Basic in-memory node store implementation. Useful as a base class for
  * more complex functionality.
  */
-public class MemoryNodeStore implements NodeStore {
+public class MemoryNodeStore implements NodeStore, Observable {
 
     private final AtomicReference<NodeState> root;
 
     private final Map<String, NodeState> checkpoints = newHashMap();
 
+    private final Map<Closeable, Observer> observers = newHashMap();
+
     public MemoryNodeStore(NodeState state) {
         this.root = new AtomicReference<NodeState>(state);
     }
@@ -70,6 +75,30 @@ public class MemoryNodeStore implements 
     }
 
     @Override
+    public synchronized Closeable addObserver(Observer observer) {
+        observer.contentChanged(getRoot(), null);
+
+        Closeable closeable = new Closeable() {
+            @Override
+            public void close() throws IOException {
+                synchronized (MemoryNodeStore.this) {
+                    observers.remove(this);
+                }
+            }
+        };
+        observers.put(closeable, observer);
+        return closeable;
+    }
+
+    private synchronized void setRoot(NodeState root, CommitInfo info) {
+        this.root.getAndSet(root);
+
+        for (Observer observer : observers.values()) {
+            observer.contentChanged(root, info);
+        }
+    }
+
+    @Override
     public NodeState getRoot() {
         return root.get();
     }
@@ -215,7 +244,7 @@ public class MemoryNodeStore implements 
             // TODO: rebase();
             checkNotMerged();
             NodeState merged = squeeze(hook.processCommit(base, root, info));
-            store.root.set(merged);
+            store.setRoot(merged, info);
             root = null; // Mark as merged
             return merged;
         }

Propchange: jackrabbit/oak/branches/1.0/oak-lucene/
------------------------------------------------------------------------------
--- svn:mergeinfo (added)
+++ svn:mergeinfo Wed Apr 16 13:46:53 2014
@@ -0,0 +1,2 @@
+/jackrabbit/oak/trunk/oak-lucene:1584578,1584602,1584614,1584616,1584709,1584781,1584937,1585297,1585304-1585305,1585420,1585424,1585427,1585448,1585465,1585468,1585486,1585497,1585509,1585647,1585655-1585656,1585661,1585665-1585666,1585669-1585670,1585673,1585680,1585719,1585763,1585770,1585896,1585904,1585907,1585940,1585949,1585951,1585956,1585962,1586287,1586320,1586364,1586372,1586655,1586836,1587130,1587224,1587399,1587408,1587428,1587472,1587485,1587504,1587538,1587580
+/jackrabbit/trunk/oak-lucene:1345480

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndex.java Wed Apr 16 13:46:53 2014
@@ -16,6 +16,7 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
+import static com.google.common.base.Preconditions.checkState;
 import static org.apache.jackrabbit.JcrConstants.JCR_MIXINTYPES;
 import static org.apache.jackrabbit.JcrConstants.JCR_PRIMARYTYPE;
 import static org.apache.jackrabbit.oak.api.Type.STRING;
@@ -24,34 +25,25 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.commons.PathUtils.getDepth;
 import static org.apache.jackrabbit.oak.commons.PathUtils.getName;
 import static org.apache.jackrabbit.oak.commons.PathUtils.getParentPath;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.INDEX_DEFINITIONS_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.IndexConstants.TYPE_PROPERTY_NAME;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.FieldNames.PATH_SELECTOR;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.EXCLUDE_PROPERTY_NAMES;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INCLUDE_PROPERTY_TYPES;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.INDEX_DATA_CHILD_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_FILE;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_NAME;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_OAK;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.PERSISTENCE_PATH;
-import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.TYPE_LUCENE;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexConstants.VERSION;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newFulltextTerm;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.TermFactory.newPathTerm;
 import static org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneIndexHelper.skipTokenization;
 import static org.apache.jackrabbit.oak.query.QueryImpl.JCR_PATH;
-import static org.apache.jackrabbit.oak.spi.query.Cursors.newPathCursor;
 import static org.apache.lucene.search.BooleanClause.Occur.MUST;
 import static org.apache.lucene.search.BooleanClause.Occur.MUST_NOT;
 import static org.apache.lucene.search.BooleanClause.Occur.SHOULD;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -60,6 +52,9 @@ import java.util.concurrent.atomic.Atomi
 
 import javax.jcr.PropertyType;
 
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Queues;
+import com.google.common.collect.Sets;
 import org.apache.jackrabbit.oak.api.PropertyValue;
 import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.MoreLikeThisHelper;
@@ -78,14 +73,11 @@ import org.apache.jackrabbit.oak.spi.que
 import org.apache.jackrabbit.oak.spi.query.PropertyValues;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex.FulltextQueryIndex;
-import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
-import org.apache.jackrabbit.oak.spi.state.ReadOnlyBuilder;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.Term;
@@ -106,8 +98,6 @@ import org.apache.lucene.search.TermQuer
 import org.apache.lucene.search.TermRangeQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.WildcardQuery;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -159,11 +149,21 @@ public class LuceneIndex implements Full
             .getLogger(LuceneIndex.class);
     public static final String NATIVE_QUERY_FUNCTION = "native*lucene";
 
+    /**
+     * Batch size for fetching results from Lucene queries.
+     */
+    static final int LUCENE_QUERY_BATCH_SIZE = 50;
+
+    private final IndexTracker tracker;
+
     private final Analyzer analyzer;
 
     private final NodeAggregator aggregator;
 
-    public LuceneIndex(Analyzer analyzer, NodeAggregator aggregator) {
+    public LuceneIndex(
+            IndexTracker tracker, Analyzer analyzer,
+            NodeAggregator aggregator) {
+        this.tracker = tracker;
         this.analyzer = analyzer;
         this.aggregator = aggregator;
     }
@@ -175,7 +175,7 @@ public class LuceneIndex implements Full
 
     @Override
     public double getCost(Filter filter, NodeState root) {
-        if (!isLive(root)) {
+        if (tracker.getIndexNode("/") == null) {
             // unusable index
             return Double.POSITIVE_INFINITY;
         }
@@ -251,79 +251,11 @@ public class LuceneIndex implements Full
         return relPaths;
     }
 
-    private static boolean isLive(NodeState root) {
-        NodeState def = getIndexDef(root);
-        if (def == null) {
-            return false;
-        }
-        String type = def.getString(PERSISTENCE_NAME);
-        if (type == null || PERSISTENCE_OAK.equalsIgnoreCase(type)) {
-            return getIndexDataNode(def) != null;
-        }
-
-        if (PERSISTENCE_FILE.equalsIgnoreCase(type)) {
-            return def.getString(PERSISTENCE_PATH) != null;
-        }
-
-        return false;
-    }
-
-    private static Directory newDirectory(NodeState root) {
-        NodeState def = getIndexDef(root);
-        if (def == null) {
-            return null;
-        }
-
-        String type = def.getString(PERSISTENCE_NAME);
-        if (type == null || PERSISTENCE_OAK.equalsIgnoreCase(type)) {
-            NodeState index = getIndexDataNode(def);
-            if (index == null) {
-                return null;
-            }
-            return new OakDirectory(new ReadOnlyBuilder(index));
-        }
-
-        if (PERSISTENCE_FILE.equalsIgnoreCase(type)) {
-            String fs = def.getString(PERSISTENCE_PATH);
-            if (fs == null) {
-                return null;
-            }
-            File f = new File(fs);
-            if (!f.exists()) {
-                return null;
-            }
-            try {
-                // TODO lock factory
-                return FSDirectory.open(f);
-            } catch (IOException e) {
-                LOG.error("Unable to open directory {}", fs);
-            }
-        }
-
-        return null;
-    }
-
-    private static NodeState getIndexDef(NodeState node) {
-        NodeState state = node.getChildNode(INDEX_DEFINITIONS_NAME);
-        for (ChildNodeEntry entry : state.getChildNodeEntries()) {
-            NodeState ns = entry.getNodeState();
-            if (TYPE_LUCENE.equals(ns.getString(TYPE_PROPERTY_NAME))) {
-                return ns;
-            }
-        }
-        return null;
-    }
-
-    private static NodeState getIndexDataNode(NodeState node) {
-        if (node.hasChildNode(INDEX_DATA_CHILD_NAME)) {
-            return node.getChildNode(INDEX_DATA_CHILD_NAME);
-        }
-        // unusable index (not initialized yet)
-        return null;
-    }
-
     @Override
     public String getPlan(Filter filter, NodeState root) {
+        IndexNode index = tracker.getIndexNode("/");
+        checkState(index != null, "The Lucene index is not available");
+
         FullTextExpression ft = filter.getFullTextConstraint();
         Set<String> relPaths = getRelativePaths(ft);
         if (relPaths.size() > 1) {
@@ -333,7 +265,7 @@ public class LuceneIndex implements Full
         // we only restrict non-full-text conditions if there is
         // no relative property in the full-text constraint
         boolean nonFullTextConstraints = parent.isEmpty();
-        String plan = getQuery(filter, null, nonFullTextConstraints, analyzer, getIndexDef(root)) + " ft:(" + ft + ")";
+        String plan = getQuery(filter, null, nonFullTextConstraints, analyzer, index.getDefinition()) + " ft:(" + ft + ")";
         if (!parent.isEmpty()) {
             plan += " parent:" + parent;
         }
@@ -341,86 +273,116 @@ public class LuceneIndex implements Full
     }
 
     @Override
-    public Cursor query(Filter filter, NodeState root) {
-        if (!isLive(root)) {
-            throw new IllegalStateException("Lucene index is not live");
-        }
+    public Cursor query(final Filter filter, final NodeState root) {
+        final IndexNode index = tracker.getIndexNode("/");
+        checkState(index != null, "The Lucene index is not available");
+
         FullTextExpression ft = filter.getFullTextConstraint();
         Set<String> relPaths = getRelativePaths(ft);
         if (relPaths.size() > 1) {
             return new MultiLuceneIndex(filter, root, relPaths).query();
         }
-        String parent = relPaths.size() == 0 ? "" : relPaths.iterator().next();
+        final String parent = relPaths.size() == 0 ? "" : relPaths.iterator().next();
         // we only restrict non-full-text conditions if there is
         // no relative property in the full-text constraint
-        boolean nonFullTextConstraints = parent.isEmpty();
-        Directory directory = newDirectory(root);
+        final boolean nonFullTextConstraints = parent.isEmpty();
+        final int parentDepth = getDepth(parent);
         QueryEngineSettings settings = filter.getQueryEngineSettings();
-        if (directory == null) {
-            return newPathCursor(Collections.<String> emptySet(), settings);
-        }
-        long s = System.currentTimeMillis();
-        try {
-            try {
-                IndexReader reader = DirectoryReader.open(directory);
+        Iterator<LuceneResultRow> itr = new AbstractIterator<LuceneResultRow>() {
+            private final Deque<LuceneResultRow> queue = Queues.newArrayDeque();
+            private final Set<String> seenPaths = Sets.newHashSet();
+            private ScoreDoc lastDoc;
+
+            @Override
+            protected LuceneResultRow computeNext() {
+                while (!queue.isEmpty() || loadDocs()) {
+                    return queue.remove();
+                }
+                return endOfData();
+            }
+
+            private LuceneResultRow convertToRow(ScoreDoc doc, IndexSearcher searcher) throws IOException {
+                String path = searcher.getIndexReader().document(doc.doc,
+                        PATH_SELECTOR).get(PATH);
+                if (path != null) {
+                    if ("".equals(path)) {
+                        path = "/";
+                    }
+                    if (!parent.isEmpty()) {
+                        // TODO OAK-828 this breaks node aggregation
+                        // get the base path
+                        // ensure the path ends with the given
+                        // relative path
+                        // if (!path.endsWith("/" + parent)) {
+                        // continue;
+                        // }
+                        path = getAncestorPath(path, parentDepth);
+                        // avoid duplicate entries
+                        if (seenPaths.contains(path)) {
+                            return null;
+                        }
+                        seenPaths.add(path);
+                    }
+
+                    return new LuceneResultRow(path, doc.score);
+                }
+                return null;
+            }
+
+            /**
+             * Loads the lucene documents in batches
+             * @return true if any document is loaded
+             */
+            private boolean loadDocs() {
+                IndexNode indexNode = null;
+                IndexSearcher searcher = null;
+                ScoreDoc lastDocToRecord = null;
                 try {
-                    IndexSearcher searcher = new IndexSearcher(reader);
-                    List<LuceneResultRow> rows = new ArrayList<LuceneResultRow>();
-                    Query query = getQuery(filter, reader,
-                            nonFullTextConstraints, analyzer, getIndexDef(root));
-
-                    // TODO OAK-828
-                    HashSet<String> seenPaths = new HashSet<String>();
-                    int parentDepth = getDepth(parent);
-                    if (query != null) {
-                        // OAK-925
-                        // TODO how to best avoid loading all entries in memory?
-                        // (memory problem and performance problem)
-                        TopDocs docs = searcher
-                                .search(query, Integer.MAX_VALUE);
-                        for (ScoreDoc doc : docs.scoreDocs) {
-                            String path = reader.document(doc.doc,
-                                    PATH_SELECTOR).get(PATH);
-                            if (path != null) {
-                                if ("".equals(path)) {
-                                    path = "/";
-                                }
-                                if (!parent.isEmpty()) {
-                                    // TODO OAK-828 this breaks node aggregation
-                                    // get the base path
-                                    // ensure the path ends with the given
-                                    // relative path
-                                    // if (!path.endsWith("/" + parent)) {
-                                    // continue;
-                                    // }
-                                    path = getAncestorPath(path, parentDepth);
-                                    // avoid duplicate entries
-                                    if (seenPaths.contains(path)) {
-                                        continue;
-                                    }
-                                    seenPaths.add(path);
-                                }
-
-                                LuceneResultRow r = new LuceneResultRow();
-                                r.path = path;
-                                r.score = doc.score;
-                                rows.add(r);
-                            }
+                    indexNode = acquire();
+                    searcher = indexNode.acquireSearcher();
+                    Query query = getQuery(filter, searcher.getIndexReader(),
+                            nonFullTextConstraints, analyzer, index.getDefinition());
+                    TopDocs docs;
+                    if (lastDoc != null) {
+                        docs = searcher.searchAfter(lastDoc, query, LUCENE_QUERY_BATCH_SIZE);
+                    } else {
+                        docs = searcher.search(query, LUCENE_QUERY_BATCH_SIZE);
+                    }
+
+                    for (ScoreDoc doc : docs.scoreDocs) {
+                        LuceneResultRow row = convertToRow(doc, searcher);
+                        if(row != null) {
+                            queue.add(row);
                         }
+                        lastDocToRecord = doc;
                     }
-                    LOG.debug("query via {} took {} ms.", this,
-                            System.currentTimeMillis() - s);
-                    return new LucenePathCursor(rows, settings);
+                } catch (IOException e) {
+                    LOG.warn("query via {} failed.", LuceneIndex.this, e);
                 } finally {
-                    reader.close();
+                    release(indexNode, searcher);
                 }
-            } finally {
-                directory.close();
+                if (lastDocToRecord != null) {
+                    this.lastDoc = lastDocToRecord;
+                }
+                return !queue.isEmpty();
             }
-        } catch (IOException e) {
-            LOG.warn("query via {} failed.", this, e);
-            return newPathCursor(Collections.<String> emptySet(), settings);
-        }
+
+            private IndexNode acquire() {
+                return tracker.getIndexNode("/");
+            }
+
+            private void release(IndexNode indexNode, IndexSearcher searcher){
+                try {
+                    if(searcher != null){
+                        indexNode.releaseSearcher();
+                    }
+                } catch (IOException e) {
+                    LOG.warn("Error occurred while releasing/closing the " +
+                            "IndexSearcher", e);
+                }
+            }
+        };
+        return new LucenePathCursor(itr, settings);
     }
 
     /**
@@ -924,8 +886,18 @@ public class LuceneIndex implements Full
     }
     
     static class LuceneResultRow {
-        String path;
-        double score;
+        final String path;
+        final double score;
+
+        LuceneResultRow(String path, double score) {
+            this.path = path;
+            this.score = score;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("%s (%1.2f)", path, score);
+        }
     }
     
     /**
@@ -937,10 +909,7 @@ public class LuceneIndex implements Full
         private final Cursor pathCursor;
         LuceneResultRow currentRow;
         
-        LucenePathCursor(List<LuceneResultRow> list, QueryEngineSettings settings) {
-            
-            final Iterator<LuceneResultRow> it = list.iterator();
-            
+        LucenePathCursor(final Iterator<LuceneResultRow> it, QueryEngineSettings settings) {
             Iterator<String> pathIterator = new Iterator<String>() {
 
                 @Override

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexProvider.java Wed Apr 16 13:46:53 2014
@@ -21,8 +21,11 @@ import java.util.List;
 import javax.annotation.Nonnull;
 
 import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
 import org.apache.felix.scr.annotations.Service;
 import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
+import org.apache.jackrabbit.oak.spi.commit.CommitInfo;
+import org.apache.jackrabbit.oak.spi.commit.Observer;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
@@ -34,28 +37,41 @@ import com.google.common.collect.Immutab
  * A provider for Lucene indexes.
  * 
  * @see LuceneIndex
- * 
  */
 @Component
-@Service(QueryIndexProvider.class)
-public class LuceneIndexProvider implements QueryIndexProvider,
-        LuceneIndexConstants {
+@Service({ QueryIndexProvider.class, Observer.class })
+public class LuceneIndexProvider implements QueryIndexProvider, Observer {
+
+    protected final IndexTracker tracker = new IndexTracker();
 
     /**
      * TODO how to inject this in an OSGi friendly way?
      */
-    protected Analyzer analyzer = ANALYZER;
+    protected Analyzer analyzer = LuceneIndexConstants.ANALYZER;
 
     protected NodeAggregator aggregator = null;
 
+    @Deactivate
+    public void close() {
+        tracker.close();
+    }
+
+    //----------------------------------------------------------< Observer >--
+
     @Override
-    @Nonnull
+    public void contentChanged(NodeState root, CommitInfo info) {
+        tracker.update(root);
+    }
+
+    //------------------------------------------------< QueryIndexProvider >--
+
+    @Override @Nonnull
     public List<QueryIndex> getQueryIndexes(NodeState nodeState) {
         return ImmutableList.<QueryIndex> of(newLuceneIndex());
     }
 
     protected LuceneIndex newLuceneIndex() {
-        return new LuceneIndex(analyzer, aggregator);
+        return new LuceneIndex(tracker, analyzer, aggregator);
     }
 
     /**

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/LuceneOakRepositoryStub.java Wed Apr 16 13:46:53 2014
@@ -29,7 +29,9 @@ import org.apache.jackrabbit.oak.plugins
 import org.apache.jackrabbit.oak.plugins.index.aggregate.SimpleNodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.lucene.LowCostLuceneIndexProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditorProvider;
+import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneInitializerHelper;
+import org.apache.jackrabbit.oak.spi.commit.Observer;
 
 public class LuceneOakRepositoryStub extends OakTarMKRepositoryStub {
 
@@ -40,10 +42,10 @@ public class LuceneOakRepositoryStub ext
 
     @Override
     protected void preCreateRepository(Jcr jcr) {
+        LuceneIndexProvider provider = new LowCostLuceneIndexProvider();
         jcr.with(new LuceneInitializerHelper("luceneGlobal", null))
-                .with(AggregateIndexProvider
-                        .wrap(new LowCostLuceneIndexProvider()
-                                .with(getNodeAggregator())))
+                .with(AggregateIndexProvider.wrap(provider.with(getNodeAggregator())))
+                .with((Observer) provider)
                 .with(new LuceneIndexEditorProvider());
     }
 

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/query/MultiSessionQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/query/MultiSessionQueryTest.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/query/MultiSessionQueryTest.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/jcr/query/MultiSessionQueryTest.java Wed Apr 16 13:46:53 2014
@@ -26,6 +26,8 @@ import org.apache.jackrabbit.oak.jcr.Jcr
 import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.LuceneIndexProvider;
 import org.apache.jackrabbit.oak.plugins.index.lucene.util.LuceneInitializerHelper;
+import org.apache.jackrabbit.oak.spi.commit.Observer;
+import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -46,7 +48,9 @@ public class MultiSessionQueryTest {
 
         // lucene specific
         jcr.with(new LuceneInitializerHelper("lucene").async());
-        jcr.with(new LuceneIndexProvider());
+        LuceneIndexProvider provider = new LuceneIndexProvider();
+        jcr.with((QueryIndexProvider) provider);
+        jcr.with((Observer) provider);
         jcr.with(new LuceneIndexEditorProvider());
 
         repository = jcr.createRepository();

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LowCostLuceneIndexProvider.java Wed Apr 16 13:46:53 2014
@@ -30,13 +30,13 @@ public class LowCostLuceneIndexProvider 
 
     @Override
     protected LuceneIndex newLuceneIndex() {
-        return new LowCostLuceneIndex(analyzer, aggregator);
+        return new LowCostLuceneIndex(tracker, analyzer, aggregator);
     }
 
     private static class LowCostLuceneIndex extends LuceneIndex {
 
-        public LowCostLuceneIndex(Analyzer analyzer, NodeAggregator aggregator) {
-            super(analyzer, aggregator);
+        public LowCostLuceneIndex(IndexTracker tracker, Analyzer analyzer, NodeAggregator aggregator) {
+            super(tracker, analyzer, aggregator);
         }
 
         @Override

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexAggregationTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexAggregationTest.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexAggregationTest.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexAggregationTest.java Wed Apr 16 13:46:53 2014
@@ -39,6 +39,7 @@ import static org.apache.jackrabbit.oak.
 
 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.security.OpenSecurityProvider;
 import org.junit.Test;
 
@@ -55,12 +56,12 @@ public class LuceneIndexAggregationTest 
 
     @Override
     protected ContentRepository createRepository() {
+        LowCostLuceneIndexProvider provider = new LowCostLuceneIndexProvider();
         return new Oak()
                 .with(new InitialContent())
                 .with(new OpenSecurityProvider())
-                .with(AggregateIndexProvider
-                        .wrap(new LowCostLuceneIndexProvider()
-                                .with(getNodeAggregator())))
+                .with(AggregateIndexProvider.wrap(provider.with(getNodeAggregator())))
+                .with((Observer) provider)
                 .with(new LuceneIndexEditorProvider())
                 .createContentRepository();
     }

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexExclusionQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexExclusionQueryTest.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexExclusionQueryTest.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexExclusionQueryTest.java Wed Apr 16 13:46:53 2014
@@ -35,6 +35,8 @@ import org.apache.jackrabbit.oak.api.Con
 import org.apache.jackrabbit.oak.api.Tree;
 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.junit.Test;
 
@@ -56,9 +58,11 @@ public class LuceneIndexExclusionQueryTe
 
     @Override
     protected ContentRepository createRepository() {
+        LowCostLuceneIndexProvider provider = new LowCostLuceneIndexProvider();
         return new Oak().with(new InitialContent())
                 .with(new OpenSecurityProvider())
-                .with(new LowCostLuceneIndexProvider())
+                .with((QueryIndexProvider) provider)
+                .with((Observer) provider)
                 .with(new LuceneIndexEditorProvider())
                 .createContentRepository();
     }

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexQueryTest.java Wed Apr 16 13:46:53 2014
@@ -17,12 +17,16 @@
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
 import com.google.common.collect.ImmutableList;
+
 import java.util.Iterator;
+
 import org.apache.jackrabbit.oak.Oak;
 import org.apache.jackrabbit.oak.api.ContentRepository;
 import org.apache.jackrabbit.oak.api.Tree;
 import org.apache.jackrabbit.oak.plugins.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.junit.Test;
 
@@ -47,9 +51,11 @@ public class LuceneIndexQueryTest extend
 
     @Override
     protected ContentRepository createRepository() {
+        LowCostLuceneIndexProvider provider = new LowCostLuceneIndexProvider();
         return new Oak().with(new InitialContent())
                 .with(new OpenSecurityProvider())
-                .with(new LowCostLuceneIndexProvider())
+                .with((QueryIndexProvider) provider)
+                .with((Observer) provider)
                 .with(new LuceneIndexEditorProvider())
                 .createContentRepository();
     }

Modified: jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java?rev=1587906&r1=1587905&r2=1587906&view=diff
==============================================================================
--- jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java (original)
+++ jackrabbit/oak/branches/1.0/oak-lucene/src/test/java/org/apache/jackrabbit/oak/plugins/index/lucene/LuceneIndexTest.java Wed Apr 16 13:46:53 2014
@@ -16,6 +16,10 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.lucene;
 
+import java.util.List;
+
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.Iterators.transform;
 import static javax.jcr.PropertyType.TYPENAME_STRING;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertFalse;
@@ -27,6 +31,7 @@ import static org.apache.jackrabbit.oak.
 import static org.apache.jackrabbit.oak.plugins.nodetype.NodeTypeConstants.JCR_NODE_TYPES;
 import static org.apache.jackrabbit.oak.plugins.nodetype.write.InitialContent.INITIAL_CONTENT;
 
+import com.google.common.base.Function;
 import org.apache.jackrabbit.oak.api.Type;
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateProvider;
 import org.apache.jackrabbit.oak.query.QueryEngineSettings;
@@ -37,6 +42,7 @@ import org.apache.jackrabbit.oak.spi.com
 import org.apache.jackrabbit.oak.spi.commit.EditorHook;
 import org.apache.jackrabbit.oak.spi.query.Cursor;
 import org.apache.jackrabbit.oak.spi.query.Filter;
+import org.apache.jackrabbit.oak.spi.query.IndexRow;
 import org.apache.jackrabbit.oak.spi.query.PropertyValues;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
@@ -71,7 +77,9 @@ public class LuceneIndexTest {
 
         NodeState indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
 
-        QueryIndex queryIndex = new LuceneIndex(analyzer, null);
+        IndexTracker tracker = new IndexTracker();
+        tracker.update(indexed);
+        QueryIndex queryIndex = new LuceneIndex(tracker, analyzer, null);
         FilterImpl filter = createFilter(NT_BASE);
         filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,
@@ -83,6 +91,40 @@ public class LuceneIndexTest {
     }
 
     @Test
+    public void testLuceneLazyCursor() throws Exception {
+        NodeBuilder index = builder.child(INDEX_DEFINITIONS_NAME);
+        newLuceneIndexDefinition(index, "lucene",
+                ImmutableSet.of(TYPENAME_STRING));
+
+        NodeState before = builder.getNodeState();
+        builder.setProperty("foo", "bar");
+
+        for(int i = 0; i < LuceneIndex.LUCENE_QUERY_BATCH_SIZE; i++){
+            builder.child("parent").child("child"+i).setProperty("foo", "bar");
+        }
+
+        NodeState after = builder.getNodeState();
+
+        NodeState indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
+
+        IndexTracker tracker = new IndexTracker();
+        tracker.update(indexed);
+        QueryIndex queryIndex = new LuceneIndex(tracker, analyzer, null);
+        FilterImpl filter = createFilter(NT_BASE);
+        filter.restrictProperty("foo", Operator.EQUAL,
+                PropertyValues.newString("bar"));
+        Cursor cursor = queryIndex.query(filter, indexed);
+
+        List<String> paths = copyOf(transform(cursor, new Function<IndexRow, String>() {
+            public String apply(IndexRow input) {
+                return input.getPath();
+            }
+        }));
+        assertTrue(!paths.isEmpty());
+        assertEquals(LuceneIndex.LUCENE_QUERY_BATCH_SIZE + 1, paths.size());
+    }
+
+    @Test
     public void testLucene2() throws Exception {
         NodeBuilder index = builder.child(INDEX_DEFINITIONS_NAME);
         newLuceneIndexDefinition(index, "lucene",
@@ -98,7 +140,9 @@ public class LuceneIndexTest {
 
         NodeState indexed = HOOK.processCommit(before, after, CommitInfo.EMPTY);
 
-        QueryIndex queryIndex = new LuceneIndex(analyzer, null);
+        IndexTracker tracker = new IndexTracker();
+        tracker.update(indexed);
+        QueryIndex queryIndex = new LuceneIndex(tracker, analyzer, null);
         FilterImpl filter = createFilter(NT_BASE);
         // filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,
@@ -130,7 +174,9 @@ public class LuceneIndexTest {
 
         NodeState indexed = HOOK.processCommit(before, after,CommitInfo.EMPTY);
 
-        QueryIndex queryIndex = new LuceneIndex(analyzer, null);
+        IndexTracker tracker = new IndexTracker();
+        tracker.update(indexed);
+        QueryIndex queryIndex = new LuceneIndex(tracker, analyzer, null);
         FilterImpl filter = createFilter(NT_BASE);
         // filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty("foo", Operator.EQUAL,



Re: svn commit: r1587906 - in /jackrabbit/oak/branches/1.0: oak-core/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/ oak-lucene/ oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ oak-lucene/src/test/java/org/apache/...

Posted by Jukka Zitting <ju...@gmail.com>.
Hi,

On Wed, Apr 16, 2014 at 3:40 PM, Alex Parvulescu
<al...@gmail.com> wrote:
> ack. the issue there was the revisions also contained some benchmark stuff
> which I didn't need to merge into 1.0.

For future reference, an alternative solution that achieves the same
effect without diverging the mergeinfos would be:

    $ svn merge ...
    $ svn revert -R oak-run    # or manually edit the merge
    $ svn commit ...

BR,

Jukka Zitting

Re: svn commit: r1587906 - in /jackrabbit/oak/branches/1.0: oak-core/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/ oak-lucene/ oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ oak-lucene/src/test/java/org/apache/...

Posted by Alex Parvulescu <al...@gmail.com>.
ack. the issue there was the revisions also contained some benchmark stuff
which I didn't need to merge into 1.0.

alex


On Wed, Apr 16, 2014 at 9:22 PM, Jukka Zitting <ju...@gmail.com>wrote:

> Hi,
>
> On Wed, Apr 16, 2014 at 9:46 AM,  <al...@apache.org> wrote:
> > Modified:
> >     jackrabbit/oak/branches/1.0/oak-core/   (props changed)
> >     jackrabbit/oak/branches/1.0/oak-lucene/   (props changed)
>
> When backporting changes, it's better to always do the merging at the
> root of the branch, as otherwise we'll have a hard time properly
> tracking the merge history with mergeinfo.
>
> I removed these subdirectory mergeinfos in revision 1588040.
>
> BR,
>
> Jukka Zitting
>

Re: svn commit: r1587906 - in /jackrabbit/oak/branches/1.0: oak-core/ oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/memory/ oak-lucene/ oak-lucene/src/main/java/org/apache/jackrabbit/oak/plugins/index/lucene/ oak-lucene/src/test/java/org/apache/...

Posted by Jukka Zitting <ju...@gmail.com>.
Hi,

On Wed, Apr 16, 2014 at 9:46 AM,  <al...@apache.org> wrote:
> Modified:
>     jackrabbit/oak/branches/1.0/oak-core/   (props changed)
>     jackrabbit/oak/branches/1.0/oak-lucene/   (props changed)

When backporting changes, it's better to always do the merging at the
root of the branch, as otherwise we'll have a hard time properly
tracking the merge history with mergeinfo.

I removed these subdirectory mergeinfos in revision 1588040.

BR,

Jukka Zitting