You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by al...@apache.org on 2012/08/01 16:06:51 UTC

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

Author: alexparvulescu
Date: Wed Aug  1 14:06:51 2012
New Revision: 1368008

URL: http://svn.apache.org/viewvc?rev=1368008&view=rev
Log:
OAK-154 Full text search index

Added:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java   (with props)
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java   (with props)
Modified:
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/core/ContentRepositoryImpl.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditor.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndex.java
    jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexProvider.java
    jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditorTest.java

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/core/ContentRepositoryImpl.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/core/ContentRepositoryImpl.java?rev=1368008&r1=1368007&r2=1368008&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/core/ContentRepositoryImpl.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/core/ContentRepositoryImpl.java Wed Aug  1 14:06:51 2012
@@ -32,6 +32,7 @@ import org.apache.jackrabbit.oak.api.Con
 import org.apache.jackrabbit.oak.api.ContentSession;
 import org.apache.jackrabbit.oak.api.QueryEngine;
 import org.apache.jackrabbit.oak.kernel.KernelNodeStore;
+import org.apache.jackrabbit.oak.plugins.lucene.LuceneEditor;
 import org.apache.jackrabbit.oak.plugins.name.NameValidatorProvider;
 import org.apache.jackrabbit.oak.plugins.name.NamespaceValidatorProvider;
 import org.apache.jackrabbit.oak.plugins.type.TypeValidatorProvider;
@@ -95,6 +96,7 @@ public class ContentRepositoryImpl imple
  
         List<CommitEditor> editors = new ArrayList<CommitEditor>();
         editors.add(new ValidatingEditor(validatorProvider));
+        editors.add(new LuceneEditor());
         nodeStore.setEditor(new CompositeEditor(editors));
 
         QueryIndexProvider qip = (indexProvider == null) ? getDefaultIndexProvider(microKernel) : indexProvider;

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditor.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditor.java?rev=1368008&r1=1368007&r2=1368008&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditor.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditor.java Wed Aug  1 14:06:51 2012
@@ -16,6 +16,11 @@
  */
 package org.apache.jackrabbit.oak.plugins.lucene;
 
+import static org.apache.jackrabbit.oak.commons.PathUtils.concat;
+import static org.apache.jackrabbit.oak.plugins.lucene.FieldFactory.newPathField;
+import static org.apache.jackrabbit.oak.plugins.lucene.FieldFactory.newPropertyField;
+import static org.apache.jackrabbit.oak.plugins.lucene.TermFactory.newPathTerm;
+
 import java.io.IOException;
 
 import javax.jcr.PropertyType;
@@ -38,10 +43,6 @@ import org.apache.lucene.util.Version;
 import org.apache.tika.Tika;
 import org.apache.tika.exception.TikaException;
 
-import static org.apache.jackrabbit.oak.plugins.lucene.FieldFactory.newPathField;
-import static org.apache.jackrabbit.oak.plugins.lucene.FieldFactory.newPropertyField;
-import static org.apache.jackrabbit.oak.plugins.lucene.TermFactory.newPathTerm;
-
 /**
  * This class updates a Lucene index when node content is changed.
  */
@@ -53,12 +54,19 @@ public class LuceneEditor implements Com
 
     private static final Analyzer ANALYZER = new StandardAnalyzer(VERSION);
 
+    private static final IndexWriterConfig config = new IndexWriterConfig(VERSION,
+            ANALYZER);
+
     private final String[] path;
 
     public LuceneEditor(String... path) {
         this.path = path;
     }
 
+    public LuceneEditor() {
+        this(LuceneIndexUtils.DEFAULT_INDEX_PATH);
+    }
+
     @Override
     public NodeState editCommit(
             NodeStore store, NodeState before, NodeState after)
@@ -66,10 +74,9 @@ public class LuceneEditor implements Com
         try {
             OakDirectory directory = new OakDirectory(store, after, path);
 
-            IndexWriter writer = new IndexWriter(
-                    directory, new IndexWriterConfig(VERSION, ANALYZER));
+            IndexWriter writer = new IndexWriter(directory, config);
             try {
-                LuceneDiff diff = new LuceneDiff(writer, "");
+                LuceneDiff diff = new LuceneDiff(writer, "/");
                 after.compareAgainstBaseState(before, diff);
                 diff.postProcess(after);
                 writer.commit();
@@ -132,7 +139,7 @@ public class LuceneEditor implements Com
             }
             if (exception == null) {
                 try {
-                    addSubtree(path + "/" + name, after);
+                    addSubtree(concat(path, name), after);
                 } catch (IOException e) {
                     exception = e;
                 }
@@ -147,7 +154,7 @@ public class LuceneEditor implements Com
             }
             if (exception == null) {
                 try {
-                    LuceneDiff diff = new LuceneDiff(writer, path + "/" + name);
+                    LuceneDiff diff = new LuceneDiff(writer, concat(path, name));
                     after.compareAgainstBaseState(before, diff);
                     diff.postProcess(after);
                 } catch (IOException e) {
@@ -163,7 +170,7 @@ public class LuceneEditor implements Com
             }
             if (exception == null) {
                 try {
-                    deleteSubtree(path + "/" + name, before);
+                    deleteSubtree(concat(path, name), before);
                 } catch (IOException e) {
                     exception = e;
                 }
@@ -174,7 +181,7 @@ public class LuceneEditor implements Com
                 throws IOException {
             writer.addDocument(makeDocument(path, state));
             for (ChildNodeEntry entry : state.getChildNodeEntries()) {
-                addSubtree(path + "/" + entry.getName(), entry.getNodeState());
+                addSubtree(concat(path, entry.getName()), entry.getNodeState());
             }
         }
 
@@ -182,7 +189,7 @@ public class LuceneEditor implements Com
                 throws IOException {
             writer.deleteDocuments(newPathTerm(path));
             for (ChildNodeEntry entry : state.getChildNodeEntries()) {
-                deleteSubtree(path + "/" + entry.getName(), entry.getNodeState());
+                deleteSubtree(concat(path, entry.getName()), entry.getNodeState());
             }
         }
 

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndex.java?rev=1368008&r1=1368007&r2=1368008&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndex.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndex.java Wed Aug  1 14:06:51 2012
@@ -53,16 +53,16 @@ public class LuceneIndex implements Quer
 
     private final NodeStore store;
 
-    private final String[] path;
+    private final LuceneIndexInfo index;
 
-    public LuceneIndex(NodeStore store, String... path) {
+    public LuceneIndex(NodeStore store, LuceneIndexInfo index) {
         this.store = store;
-        this.path = path;
+        this.index = index;
     }
 
     @Override
     public String getIndexName() {
-        return "lucene";
+        return index.getName();
     }
 
     @Override
@@ -78,7 +78,7 @@ public class LuceneIndex implements Quer
     @Override
     public Cursor query(Filter filter, String revisionId) {
         try {
-            Directory directory = new OakDirectory(store, store.getRoot(), path);
+            Directory directory = new OakDirectory(store, store.getRoot(), index.getPath());
             try {
                 IndexReader reader = DirectoryReader.open(directory);
                 try {
@@ -142,8 +142,16 @@ public class LuceneIndex implements Quer
 
         for (PropertyRestriction pr : filter.getPropertyRestrictions()) {
             String name = pr.propertyName;
-            String first = pr.first.getString();
-            String last = pr.last.getString();
+            String first = null;
+            String last = null;
+
+            if (pr.first != null) {
+                first = pr.first.getString();
+            }
+            if (pr.last != null) {
+                last = pr.last.getString();
+            }
+
             if (first .equals(last) && pr.firstIncluding && pr.lastIncluding) {
                 qs.add(new TermQuery(new Term(name, first)));
             } else {
@@ -159,7 +167,7 @@ public class LuceneIndex implements Quer
             }
             return bq;
         } else {
-            return qs.get(1);
+            return qs.get(0);
         }
     }
 

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java?rev=1368008&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java Wed Aug  1 14:06:51 2012
@@ -0,0 +1,61 @@
+/*
+ * 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.lucene;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * LuceneIndexInfo contains information about a lucene index
+ * 
+ */
+public class LuceneIndexInfo {
+
+    /**
+     * the index name
+     */
+    private final String name;
+
+    /**
+     * the index path, broken into path segments
+     */
+    private final String[] path;
+
+    public LuceneIndexInfo(String name, String[] path) {
+        this.name = name;
+        this.path = path;
+    }
+
+    public LuceneIndexInfo(String name, List<String> path) {
+        this(name, path.toArray(new String[path.size()]));
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String[] getPath() {
+        return path;
+    }
+
+    @Override
+    public String toString() {
+        return "IndexInfo [name=" + name + ", path=" + Arrays.toString(path)
+                + "]";
+    }
+
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexInfo.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexProvider.java?rev=1368008&r1=1368007&r2=1368008&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexProvider.java Wed Aug  1 14:06:51 2012
@@ -16,14 +16,21 @@
  */
 package org.apache.jackrabbit.oak.plugins.lucene;
 
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.getIndexInfos;
+
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
 import org.apache.jackrabbit.mk.api.MicroKernel;
+import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.kernel.KernelNodeStore;
 import org.apache.jackrabbit.oak.spi.QueryIndex;
 import org.apache.jackrabbit.oak.spi.QueryIndexProvider;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
 import org.apache.jackrabbit.oak.spi.state.NodeStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A provider for Lucene indexes. There is exactly one Lucene index instance per
@@ -31,10 +38,68 @@ import org.apache.jackrabbit.oak.spi.sta
  */
 public class LuceneIndexProvider implements QueryIndexProvider {
 
-    @Override
-    public List<QueryIndex> getQueryIndexes(MicroKernel mk) {
+    private static final Logger LOG = LoggerFactory
+            .getLogger(LuceneIndexProvider.class);
+
+    private final String indexPath;
+
+    private boolean init;
+
+    /**
+     * The indexes list
+     * 
+     * lazy init
+     */
+    private List<QueryIndex> indexes = null;
+
+    public LuceneIndexProvider(String indexPath) {
+        this.indexPath = indexPath;
+    }
+
+    private void init(MicroKernel mk) {
+        if (init) {
+            return;
+        }
+        LOG.debug("initializing indexes");
+
+        if (!PathUtils.isValid(indexPath)) {
+            LOG.warn("index path is not valid {}", indexPath);
+            indexes = Collections.<QueryIndex> emptyList();
+            init = true;
+            return;
+        }
+
         NodeStore store = new KernelNodeStore(mk);
-        return Collections.<QueryIndex>singletonList(new LuceneIndex(store));
+        NodeState index = store.getRoot();
+        for (String e : PathUtils.elements(indexPath)) {
+            if (PathUtils.denotesRoot(e)) {
+                continue;
+            }
+            index = index.getChildNode(e);
+            if (index == null) {
+                break;
+            }
+        }
+
+        if (index == null) {
+            // TODO what should happen when the index node doesn't exist?
+            indexes = Collections.<QueryIndex> emptyList();
+            init = true;
+            return;
+        }
+
+        List<QueryIndex> tempIndexes = new ArrayList<QueryIndex>();
+        for (LuceneIndexInfo childIndex : getIndexInfos(index, indexPath)) {
+            LOG.debug("adding a new lucene index instance @ {}", childIndex);
+            tempIndexes.add(new LuceneIndex(store, childIndex));
+        }
+        indexes = new ArrayList<QueryIndex>(tempIndexes);
+        init = true;
     }
 
+    @Override
+    public List<QueryIndex> getQueryIndexes(MicroKernel mk) {
+        init(mk);
+        return indexes;
+    }
 }

Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java?rev=1368008&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java Wed Aug  1 14:06:51 2012
@@ -0,0 +1,108 @@
+/*
+ * 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.lucene;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.jackrabbit.oak.api.Tree;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+public class LuceneIndexUtils {
+
+    /**
+     * switch to "oak:index" as soon as it is possible
+     */
+    public static final String DEFAULT_INDEX_HOME = "/oak-index";
+
+    public static final String DEFAULT_INDEX_NAME = "default";
+
+    public static final String[] DEFAULT_INDEX_PATH = { "oak-index", "default",
+            ":data" };
+
+    private LuceneIndexUtils() {
+
+    }
+
+    /**
+     * 
+     * You still need to call #commit afterwards to persist the changes
+     * 
+     * @param index
+     * @param path
+     * @param indexName
+     * @return
+     */
+    public static Tree createIndexNode(Tree index, String path, String indexName) {
+        for (String e : PathUtils.elements(path)) {
+            if (PathUtils.denotesRoot(e)) {
+                continue;
+            }
+            if (index.hasChild(e)) {
+                index = index.getChild(e);
+            } else {
+                index = index.addChild(e);
+            }
+        }
+        if (!index.hasChild(":data")) {
+            index.addChild(":data");
+        }
+        return index;
+    }
+
+    /**
+     * 
+     * Checks if any of the index's children qualifies as an index node, and
+     * returns the list of good candidates.
+     * 
+     * For now each child that has a :data node is considered to be a potential
+     * index
+     * 
+     * @param indexHome
+     *            the location of potential index nodes
+     * @return the list of existing indexes
+     */
+    public static List<LuceneIndexInfo> getIndexInfos(NodeState indexHome,
+            String parentPath) {
+        if (indexHome == null) {
+            return Collections.<LuceneIndexInfo> emptyList();
+        }
+        List<String> parent = segmentPath(parentPath);
+        List<LuceneIndexInfo> tempIndexes = new ArrayList<LuceneIndexInfo>();
+        for (ChildNodeEntry c : indexHome.getChildNodeEntries()) {
+            NodeState child = c.getNodeState();
+            if (child.hasChildNode(":data")) {
+                List<String> childIndexPath = new ArrayList<String>(parent);
+                childIndexPath.add(c.getName());
+                childIndexPath.add(":data");
+                tempIndexes.add(new LuceneIndexInfo(c.getName(), childIndexPath));
+            }
+        }
+        return tempIndexes;
+    }
+
+    private static List<String> segmentPath(String path) {
+        List<String> pathElements = new ArrayList<String>();
+        for (String e : PathUtils.elements(path)) {
+            pathElements.add(e);
+        }
+        return pathElements;
+    }
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneIndexUtils.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java?rev=1368008&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java Wed Aug  1 14:06:51 2012
@@ -0,0 +1,88 @@
+/*
+ * 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.lucene;
+
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.DEFAULT_INDEX_HOME;
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.DEFAULT_INDEX_NAME;
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.createIndexNode;
+
+import java.text.ParseException;
+
+import javax.jcr.GuestCredentials;
+
+import org.apache.jackrabbit.mk.core.MicroKernelImpl;
+import org.apache.jackrabbit.oak.api.ContentRepository;
+import org.apache.jackrabbit.oak.api.ContentSession;
+import org.apache.jackrabbit.oak.api.CoreValueFactory;
+import org.apache.jackrabbit.oak.api.QueryEngine;
+import org.apache.jackrabbit.oak.api.Result;
+import org.apache.jackrabbit.oak.api.Root;
+import org.apache.jackrabbit.oak.api.Tree;
+import org.apache.jackrabbit.oak.core.ContentRepositoryImpl;
+import org.apache.jackrabbit.oak.core.DefaultConflictHandler;
+import org.junit.Before;
+
+/**
+ * base class for lucene search tests
+ */
+public abstract class AbstractLuceneQueryTest {
+
+    private static String TEST_INDEX_NAME = DEFAULT_INDEX_NAME;
+
+    protected static String SQL2 = "JCR-SQL2";
+
+    protected ContentRepository repository;
+    protected ContentSession session;
+    protected CoreValueFactory vf;
+    protected QueryEngine qe;
+    protected Root root;
+
+    @Before
+    public void before() throws Exception {
+        repository = new ContentRepositoryImpl(new MicroKernelImpl(),
+                new LuceneIndexProvider(DEFAULT_INDEX_HOME), null);
+        session = repository.login(new GuestCredentials(), null);
+        cleanupIndexNode();
+
+        vf = session.getCoreValueFactory();
+        qe = session.getQueryEngine();
+
+    }
+
+    /**
+     * Recreates an empty index node, ready to be used in tests
+     * 
+     * @throws Exception
+     */
+    private void cleanupIndexNode() throws Exception {
+        root = session.getCurrentRoot();
+        Tree index = root.getTree(DEFAULT_INDEX_HOME);
+        if (index != null) {
+            index = index.getChild(TEST_INDEX_NAME);
+            if (index != null) {
+                index.remove();
+            }
+        }
+        createIndexNode(root.getTree("/"), DEFAULT_INDEX_HOME, TEST_INDEX_NAME);
+        root.commit(DefaultConflictHandler.OURS);
+    }
+
+    protected Result executeQuery(String statement) throws ParseException {
+        return qe.executeQuery(statement, SQL2, session, Long.MAX_VALUE, 0,
+                null, null);
+    }
+}
\ No newline at end of file

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/AbstractLuceneQueryTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditorTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditorTest.java?rev=1368008&r1=1368007&r2=1368008&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditorTest.java (original)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/LuceneEditorTest.java Wed Aug  1 14:06:51 2012
@@ -19,6 +19,8 @@ package org.apache.jackrabbit.oak.plugin
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertFalse;
 import static junit.framework.Assert.assertTrue;
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.DEFAULT_INDEX_NAME;
+import static org.apache.jackrabbit.oak.plugins.lucene.LuceneIndexUtils.DEFAULT_INDEX_PATH;
 
 import org.apache.jackrabbit.mk.core.MicroKernelImpl;
 import org.apache.jackrabbit.oak.api.Root;
@@ -38,15 +40,18 @@ public class LuceneEditorTest {
 
     @Test
     public void testLucene() throws Exception {
+        LuceneIndexInfo indexInfo = new LuceneIndexInfo(DEFAULT_INDEX_NAME,
+                DEFAULT_INDEX_PATH);
+
         KernelNodeStore store = new KernelNodeStore(new MicroKernelImpl());
-        store.setEditor(new LuceneEditor("jcr:system", "oak:lucene"));
+        store.setEditor(new LuceneEditor(indexInfo.getPath()));
         Root root = new RootImpl(store, "");
         Tree tree = root.getTree("/");
 
         tree.setProperty("foo", MemoryValueFactory.INSTANCE.createValue("bar"));
         root.commit(DefaultConflictHandler.OURS);
 
-        QueryIndex index = new LuceneIndex(store, "jcr:system", "oak:lucene");
+        QueryIndex index = new LuceneIndex(store, indexInfo);
         FilterImpl filter = new FilterImpl(null);
         filter.restrictPath("/", Filter.PathRestriction.EXACT);
         filter.restrictProperty(

Added: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java?rev=1368008&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java (added)
+++ jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java Wed Aug  1 14:06:51 2012
@@ -0,0 +1,49 @@
+/*
+ * 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.lucene;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+import java.util.Iterator;
+
+import org.apache.jackrabbit.oak.api.ResultRow;
+import org.apache.jackrabbit.oak.api.Tree;
+import org.apache.jackrabbit.oak.core.DefaultConflictHandler;
+import org.junit.Test;
+
+public class Sql2QueryTest extends AbstractLuceneQueryTest {
+
+    @Test
+    public void simpleSql2() throws Exception {
+
+        Tree test = root.getTree("/").addChild("test");
+        test.addChild("a").setProperty("name", vf.createValue("hello"));
+        test.addChild("b").setProperty("name", vf.createValue("nothello"));
+        root.commit(DefaultConflictHandler.OURS);
+
+        String sql = "select * from [nt:base] where name = 'hello'";
+
+        Iterator<? extends ResultRow> result;
+        result = executeQuery(sql).getRows().iterator();
+        assertTrue(result.hasNext());
+        assertEquals("/test/a", result.next().getPath());
+        assertFalse(result.hasNext());
+    }
+
+}

Propchange: jackrabbit/oak/trunk/oak-core/src/test/java/org/apache/jackrabbit/oak/plugins/lucene/Sql2QueryTest.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain