You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jackrabbit.apache.org by mr...@apache.org on 2005/01/11 16:17:23 UTC

svn commit: r124906 - in incubator/jackrabbit/trunk: applications/test src/conf src/java/org/apache/jackrabbit/core src/java/org/apache/jackrabbit/core/config src/java/org/apache/jackrabbit/core/search src/java/org/apache/jackrabbit/core/search/lucene

Author: mreutegg
Date: Tue Jan 11 07:17:21 2005
New Revision: 124906

URL: http://svn.apache.org/viewcvs?view=rev&rev=124906
Log:
Redesigned query manager to support pluggable implementation for search index.
Added:
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/AbstractQueryHandler.java   (contents, props changed)
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryHandler.java   (contents, props changed)
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NamespaceMappings.java   (contents, props changed)
Removed:
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/NamespaceMappings.java
Modified:
   incubator/jackrabbit/trunk/applications/test/repository.xml
   incubator/jackrabbit/trunk/src/conf/repository.xml
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java
   incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java

Modified: incubator/jackrabbit/trunk/applications/test/repository.xml
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/applications/test/repository.xml?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/applications/test/repository.xml&r1=124905&p2=incubator/jackrabbit/trunk/applications/test/repository.xml&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/applications/test/repository.xml	(original)
+++ incubator/jackrabbit/trunk/applications/test/repository.xml	Tue Jan 11 07:17:21 2005
@@ -72,9 +72,12 @@
 
     <!--
         the SearchIndex element specifies the locaction of the search index
-        (used by the QueryManager)
+        (used by the QueryHandler); the class attribute specifies the
+        FQN of the class implementing the QueryHandler interface.
     -->
     <!ELEMENT SearchIndex (param*,FileSystem)>
+    <!ATTLIST SearchIndex
+      class CDATA #REQUIRED>
 ]>
 <Repository>
     <!--
@@ -112,7 +115,7 @@
         <!--
             Search index and the file system it uses.
         -->
-        <SearchIndex>
+        <SearchIndex class="org.apache.jackrabbit.core.search.lucene.SearchIndex">
             <param name="useCompoundFile" value="true"/>
             <param name="minMergeDocs" value="1000"/>
             <param name="maxMergeDocs" value="10000"/>

Modified: incubator/jackrabbit/trunk/src/conf/repository.xml
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/conf/repository.xml?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/conf/repository.xml&r1=124905&p2=incubator/jackrabbit/trunk/src/conf/repository.xml&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/conf/repository.xml	(original)
+++ incubator/jackrabbit/trunk/src/conf/repository.xml	Tue Jan 11 07:17:21 2005
@@ -72,9 +72,12 @@
 
     <!--
         the SearchIndex element specifies the locaction of the search index
-        (used by the QueryManager)
+        (used by the QueryHandler); the class attribute specifies the
+        FQN of the class implementing the QueryHandler interface.
     -->
     <!ELEMENT SearchIndex (param*,FileSystem)>
+    <!ATTLIST SearchIndex
+      class CDATA #REQUIRED>
 ]>
 <!-- Example Repository Configuration File -->
 <Repository>
@@ -127,7 +130,7 @@
         <!--
             Search index and the file system it uses.
         -->
-        <SearchIndex>
+        <SearchIndex class="org.apache.jackrabbit.core.search.lucene.SearchIndex">
             <param name="useCompoundFile" value="true"/>
             <param name="minMergeDocs" value="1000"/>
             <param name="maxMergeDocs" value="10000"/>

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/RepositoryImpl.java	Tue Jan 11 07:17:21 2005
@@ -924,13 +924,7 @@
                     // no search index configured
                     return null;
                 }
-                try {
-                    searchMgr = new SearchManager(getSystemSession(), config.getSearchConfig());
-                } catch (IOException ioe) {
-                    String msg = "failed to instantiate search manager";
-                    log.error(msg, ioe);
-                    throw new RepositoryException(msg, ioe);
-                }
+                searchMgr = new SearchManager(getSystemSession(), config.getSearchConfig());
             }
             return searchMgr;
         }

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/SearchManager.java	Tue Jan 11 07:17:21 2005
@@ -17,29 +17,22 @@
 package org.apache.jackrabbit.core;
 
 import org.apache.jackrabbit.core.fs.FileSystem;
-import org.apache.jackrabbit.core.fs.FileSystemResource;
 import org.apache.jackrabbit.core.fs.FileSystemException;
 import org.apache.jackrabbit.core.observation.EventImpl;
 import org.apache.jackrabbit.core.observation.SynchronousEventListener;
-import org.apache.jackrabbit.core.search.NamespaceMappings;
-import org.apache.jackrabbit.core.search.OrderQueryNode;
-import org.apache.jackrabbit.core.search.QueryRootNode;
-import org.apache.jackrabbit.core.search.lucene.*;
+import org.apache.jackrabbit.core.search.QueryHandler;
 import org.apache.jackrabbit.core.state.ItemStateException;
-import org.apache.jackrabbit.core.state.ItemStateProvider;
 import org.apache.jackrabbit.core.state.NodeState;
 import org.apache.jackrabbit.core.config.SearchConfig;
 import org.apache.log4j.Logger;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Hits;
-import org.apache.lucene.search.Query;
 import org.apache.commons.collections.BeanMap;
 
 import javax.jcr.RepositoryException;
 import javax.jcr.NamespaceException;
 import javax.jcr.NamespaceRegistry;
+import javax.jcr.ItemNotFoundException;
+import javax.jcr.query.InvalidQueryException;
+import javax.jcr.query.Query;
 import javax.jcr.observation.EventIterator;
 import javax.jcr.observation.Event;
 import java.io.IOException;
@@ -53,9 +46,6 @@
     /** Logger instance for this class */
     private static final Logger log = Logger.getLogger(SearchManager.class);
 
-    /** Name of the file to persist search internal namespace mappings */
-    private static final String NS_MAPPING_FILE = "ns_mappings.properties";
-
     /** Namespace URI for xpath functions */
     // @todo this is not final! What should we use?
     private static final String NS_FN_PREFIX = "fn";
@@ -65,12 +55,6 @@
     private static final String NS_XS_PREFIX = "xs";
     public static final String NS_XS_URI = "http://www.w3.org/2001/XMLSchema";
 
-    /** The actual search index */
-    private final SearchIndex index;
-
-    /** State manager to retrieve content */
-    private final ItemStateProvider stateProvider;
-
     /** HierarchyManager for path resolution */
     private final HierarchyManager hmgr;
 
@@ -80,18 +64,14 @@
     /** Storage for search index */
     private final FileSystem fs;
 
-    /** Namespace resolver for search internal prefixes */
-    private final NamespaceMappings nsMappings;
+    /** QueryHandler where query execution is delegated to */
+    private final QueryHandler handler;
 
     public SearchManager(SessionImpl session, SearchConfig config)
-            throws RepositoryException, IOException {
+            throws RepositoryException {
         this.session = session;
-        this.stateProvider = session.getItemStateManager();
         this.hmgr = session.getHierarchyManager();
         this.fs = config.getFileSystem();
-        index = new SearchIndex(fs, new StandardAnalyzer());
-        FileSystemResource mapFile = new FileSystemResource(fs, NS_MAPPING_FILE);
-        nsMappings = new NamespaceMappings(mapFile);
 
         // register namespaces
         NamespaceRegistry nsReg = session.getWorkspace().getNamespaceRegistry();
@@ -108,8 +88,17 @@
             nsReg.registerNamespace(NS_FN_PREFIX, NS_FN_URI);
         }
 
+        // initialize query handler
+        try {
+            Class handlerClass = Class.forName(config.getHandlerClassName());
+            handler = (QueryHandler) handlerClass.newInstance();
+            handler.init(fs, session.getItemStateManager());
+        } catch (Exception e) {
+            throw new RepositoryException(e.getMessage(), e);
+        }
+
         // set properties
-        BeanMap bm = new BeanMap(this);
+        BeanMap bm = new BeanMap(handler);
         try {
             bm.putAll(config.getParameters());
         } catch (IllegalArgumentException e) {
@@ -121,15 +110,15 @@
      * Adds a <code>Node</code> to the search index.
      * @param node the NodeState to add.
      * @param path the path of the node.
-     * @throws IOException if an error occurs while adding the node to
-     * the search index.
+     * @throws RepositoryException if an error occurs while indexing the node.
+     * @throws IOException if an error occurs while adding the node to the index.
      */
-    public void addNode(NodeState node, Path path) throws IOException {
+    public void addNode(NodeState node, Path path)
+            throws RepositoryException, IOException {
         if (log.isDebugEnabled()) {
             log.debug("add node to index: " + path);
         }
-        Document doc = NodeIndexer.createDocument(node, stateProvider, path, nsMappings);
-        index.addDocument(doc);
+        handler.addNode(node);
     }
 
     /**
@@ -142,69 +131,66 @@
         if (log.isDebugEnabled()) {
             log.debug("remove node from index: " + path.toString());
         }
-        index.removeDocument(new Term(FieldNames.UUID, uuid));
+        handler.deleteNode(uuid);
     }
 
     /**
      * Closes this <code>SearchManager</code> and also closes the
-     * {@link org.apache.jackrabbit.core.fs.FileSystem} passed in the
-     * constructor of this <code>SearchManager</code>.
+     * {@link org.apache.jackrabbit.core.fs.FileSystem} configured in
+     * {@link org.apache.jackrabbit.core.config.SearchConfig}.
      */
     public void close() {
-        index.close();
         try {
+            handler.close();
             fs.close();
+        } catch (IOException e) {
+            log.error("Exception closing QueryHandler.", e);
         } catch (FileSystemException e) {
             log.error("Exception closing FileSystem.", e);
         }
     }
 
-    public QueryResultImpl execute(ItemManager itemMgr,
-                                   QueryRootNode root,
-                                   SessionImpl session)
-            throws RepositoryException {
-
-        // build lucene query
-        Query query = LuceneQueryBuilder.createQuery(root,
-                session, nsMappings, index.getAnalyzer());
-
-        OrderQueryNode orderNode = root.getOrderNode();
-        // FIXME according to spec this should be descending
-        // by default. this contrasts to standard sql semantics
-        // where default is ascending.
-        boolean[] orderSpecs = null;
-        String[] orderProperties = null;
-        if (orderNode != null) {
-            orderProperties = orderNode.getOrderByProperties();
-            orderSpecs = orderNode.getOrderBySpecs();
-        } else {
-            orderProperties = new String[0];
-            orderSpecs = new boolean[0];
-        }
-
-
-        List uuids;
-        AccessManagerImpl accessMgr = session.getAccessManager();
-
-        // execute it
-        try {
-            Hits result = index.executeQuery(query, orderProperties, orderSpecs);
-            uuids = new ArrayList(result.length());
-            for (int i = 0; i < result.length(); i++) {
-                String uuid = result.doc(i).get(FieldNames.UUID);
-                // check access
-                if (accessMgr.isGranted(new NodeId(uuid), AccessManager.READ)) {
-                    uuids.add(uuid);
-                }
-            }
-        } catch (IOException e) {
-            uuids = Collections.EMPTY_LIST;
-        }
-
-        // return QueryResult
-        return new QueryResultImpl(itemMgr,
-                (String[]) uuids.toArray(new String[uuids.size()]),
-                root.getSelectProperties());
+    /**
+     * Creates a query object that can be executed on the workspace.
+     *
+     * @param session the session of the user executing the query.
+     * @param itemMgr the item manager of the user executing the query. Needed
+     *   to return <code>Node</code> instances in the result set.
+     * @param statement the actual query statement.
+     * @param language the syntax of the query statement.
+     * @return a <code>Query</code> instance to execute.
+     *
+     * @throws InvalidQueryException if the query is malformed or the
+     *   <code>language</code> is unknown.
+     * @throws RepositoryException if any other error occurs.
+     */
+    public Query createQuery(SessionImpl session,
+                             ItemManager itemMgr,
+                             String statement,
+                             String language)
+            throws InvalidQueryException, RepositoryException {
+        return handler.createQuery(session, itemMgr, statement, language);
+    }
+    
+    /**
+     * Creates a query object from a node that can be executed on the workspace.
+     *
+     * @param session the session of the user executing the query.
+     * @param itemMgr the item manager of the user executing the query. Needed
+     *   to return <code>Node</code> instances in the result set.
+     * @param absPath absolute path to a node of type nt:query.
+     * @return a <code>Query</code> instance to execute.
+     *
+     * @throws InvalidQueryException if <code>absPath</code> is not a valid
+     *   persisted query (that is, a node of type nt:query)
+     * @throws ItemNotFoundException if there is no node at <code>absPath</code>.
+     * @throws RepositoryException if any other error occurs.
+     */
+    public Query createQuery(SessionImpl session,
+                             ItemManager itemMgr,
+                             String absPath)
+            throws InvalidQueryException, ItemNotFoundException, RepositoryException {
+        return handler.createQuery(session, itemMgr, absPath);
     }
 
     //---------------< EventListener interface >--------------------------------
@@ -225,6 +211,7 @@
                 long type = e.getType();
                 if (type == Event.NODE_ADDED) {
 
+                    // @todo use UUIDs for pending nodes?
                     Path path = Path.create(e.getPath(),
                             session.getNamespaceResolver(),
                             true);
@@ -276,8 +263,7 @@
             try {
                 Path path = (Path) it.next();
                 ItemId id = hmgr.resolvePath(path);
-                path = getIndexlessPath(path);
-                addNode((NodeState) stateProvider.getItemState(id), path);
+                addNode((NodeState) session.getItemStateManager().getItemState(id), path);
             } catch (ItemStateException e) {
                 log.error("error indexing node.", e);
             } catch (RepositoryException e) {
@@ -293,56 +279,4 @@
         }
     }
 
-    //---------------------< properties >---------------------------------------
-
-    public void setUseCompoundFile(boolean b) {
-        index.setUseCompoundFile(b);
-    }
-
-    public void setMinMergeDocs(int minMergeDocs) {
-        index.setMinMergeDocs(minMergeDocs);
-    }
-
-    public void setMaxMergeDocs(int maxMergeDocs) {
-        index.setMaxMergeDocs(maxMergeDocs);
-    }
-
-    public void setMergeFactor(int mergeFactor) {
-        index.setMergeFactor(mergeFactor);
-    }
-
-    //-----------------------< internal >---------------------------------------
-
-    /**
-     * Returns a <code>Path</code>, which contains the same sequence of path
-     * elements as <code>p</code>, but has cut off any existing indexes on the
-     * path elements.
-     *
-     * @param p the source path, possibly containing indexed path elements.
-     * @return a <code>Path</code> without indexed path elements.
-     */
-    private Path getIndexlessPath(Path p) {
-        boolean hasIndexes = false;
-        Path.PathElement[] elements = p.getElements();
-        for (int i = 0; i < elements.length && !hasIndexes; i++) {
-            hasIndexes = (elements[i].getIndex() > 0);
-        }
-
-        if (hasIndexes) {
-            // create Path without indexes
-            Path.PathBuilder builder = new Path.PathBuilder();
-            builder.addRoot();
-            for (int i = 1; i < elements.length; i++) {
-                builder.addLast(elements[i].getName());
-            }
-            try {
-                return builder.getPath();
-            } catch (MalformedPathException e) {
-                // will never happen, because Path p is always valid
-                log.error("internal error: malformed path.", e);
-            }
-        }
-        // return original path if it does not contain indexed path elements
-        return p;
-    }
 }

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/config/SearchConfig.java	Tue Jan 11 07:17:21 2005
@@ -31,6 +31,10 @@
  */
 public class SearchConfig {
 
+    /** FQN of the default query handler implementation */
+    private static final String DEFAULT_QUERY_HANDLER
+            = "org.apache.jackrabbit.core.search.lucene.SearchIndex";
+
     /** The <code>FileSystem</code> for the search index. */
     private final FileSystem fs;
 
@@ -38,6 +42,12 @@
     private Map params = new HashMap();
 
     /**
+     * The FQN of the class implementing the
+     * {@link org.apache.jackrabbit.core.search.QueryHandler} interface.
+     */
+    private final String handlerClassName;
+
+    /**
      * Creates a new <code>SearchConfig</code>.
      * @param config the config root element for this <code>SearchConfig</code>.
      * @param vars map of variable values.
@@ -60,6 +70,10 @@
         }
         // seal
         params = Collections.unmodifiableMap(params);
+
+        // handler class name
+        handlerClassName = config.getAttributeValue(AbstractConfig.CLASS_ATTRIB, 
+                DEFAULT_QUERY_HANDLER);
     }
 
     /**
@@ -77,6 +91,16 @@
      */
     public FileSystem getFileSystem() {
         return fs;
+    }
+
+    /**
+     * Returns the name of the class implementing the <code>QueryHandler</code>
+     * interface.
+     * @return the name of the class implementing the <code>QueryHandler</code>
+     *   interface.
+     */
+    public String getHandlerClassName() {
+        return handlerClassName;
     }
 
 }

Added: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/AbstractQueryHandler.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/AbstractQueryHandler.java?view=auto&rev=124906
==============================================================================
--- (empty file)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/AbstractQueryHandler.java	Tue Jan 11 07:17:21 2005
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2004-2005 The Apache Software Foundation or its licensors,
+ *                     as applicable.
+ *
+ * Licensed 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.core.search;
+
+import org.apache.jackrabbit.core.state.ItemStateProvider;
+import org.apache.jackrabbit.core.fs.FileSystem;
+
+import java.io.IOException;
+
+/**
+ * Implements default behaviour for some methods of {@link QueryHandler}.
+ */
+public abstract class AbstractQueryHandler implements QueryHandler {
+
+    /** A <code>FileSystem</code> to store the search index */
+    private FileSystem fs;
+
+    /** The persistent <code>ItemStateProvider</code> */
+    private ItemStateProvider stateProvider;
+
+    /**
+     * Initializes this query handler by setting all properties in this class
+     * with appropriate parameter values.
+     *
+     * @param fs a {@link org.apache.jackrabbit.core.fs.FileSystem} this
+     *  <code>QueryHandler</code> may use to store its index.
+     * @param stateProvider provides persistent item states.
+     */
+    public final void init(FileSystem fs, ItemStateProvider stateProvider)
+            throws IOException {
+        this.fs = fs;
+        this.stateProvider = stateProvider;
+        doInit();
+    }
+
+    /**
+     * This method must be implemented by concrete sub classes and will be
+     * called from {@link #init}.
+     */
+    protected abstract void doInit() throws IOException;
+
+    /**
+     * Returns the persistent {@link org.apache.jackrabbit.core.state.ItemStateProvider}
+     * of the workspace this <code>QueryHandler</code> is based on.
+     *
+     * @return the persistent <code>ItemStateProvider</code> of the current
+     *   workspace.
+     */
+    protected ItemStateProvider getItemStateProvider() {
+        return stateProvider;
+    }
+
+    /**
+     * Returns the {@link org.apache.jackrabbit.core.fs.FileSystem} instance
+     * this <code>QueryHandler</code> may use to store its index.
+     * @return the <code>FileSystem</code> instance for this
+     *   <code>QueryHandler</code>.
+     */
+    protected FileSystem getFileSystem() {
+        return fs;
+    }
+}

Deleted: /incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/NamespaceMappings.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/NamespaceMappings.java?view=auto&rev=124905
==============================================================================

Added: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryHandler.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryHandler.java?view=auto&rev=124906
==============================================================================
--- (empty file)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryHandler.java	Tue Jan 11 07:17:21 2005
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2004-2005 The Apache Software Foundation or its licensors,
+ *                     as applicable.
+ *
+ * Licensed 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.core.search;
+
+import org.apache.jackrabbit.core.state.NodeState;
+import org.apache.jackrabbit.core.state.ItemStateProvider;
+import org.apache.jackrabbit.core.SessionImpl;
+import org.apache.jackrabbit.core.ItemManager;
+import org.apache.jackrabbit.core.fs.FileSystem;
+
+import javax.jcr.query.Query;
+import javax.jcr.query.InvalidQueryException;
+import javax.jcr.RepositoryException;
+import javax.jcr.ItemNotFoundException;
+import java.io.IOException;
+
+/**
+ * Defines an interface for the actual node indexing and query execution.
+ * The goal is to allow different implementations based on the persistent
+ * manager in use. Some persistent model might allow to execute a query
+ * in an optimized manner, e.g. database persistence.
+ */
+public interface QueryHandler {
+
+    /**
+     * Initializes this query handler. This method is called after the
+     * <code>QueryHandler</code> is instantiated.
+     *
+     * @param fs a {@link org.apache.jackrabbit.core.fs.FileSystem} this
+     *  <code>QueryHandler</code> may use to store its index.
+     * @param stateProvider provides persistent item states.
+     * @throws IOException if an error occurs during initialization.
+     */
+    public void init(FileSystem fs, ItemStateProvider stateProvider) throws IOException;
+
+    /**
+     * Adds a <code>Node</code> to the search index.
+     * @param node the NodeState to add.
+     * @throws RepositoryException if an error occurs while indexing the node.
+     * @throws IOException if an error occurs while adding the node to the index.
+     */
+    public void addNode(NodeState node) throws RepositoryException, IOException;
+
+    /**
+     * Deletes the Node with <code>UUID</code> from the search index.
+     * @param uuid the <code>UUID</code> of the node to delete.
+     * @throws IOException if an error occurs while deleting the node.
+     */
+    public void deleteNode(String uuid) throws IOException;
+
+    /**
+     * Closes this <code>QueryHandler</code> and frees resources attached
+     * to this handler.
+     */
+    public void close() throws IOException;
+
+    /**
+     * Creates a new query by specifying the query statement itself and the
+     * language in which the query is stated.  If the query statement is
+     * syntactically invalid, given the language specified, an
+     * InvalidQueryException is thrown. <code>language</code> must specify a query language
+     * string from among those returned by QueryManager.getSupportedQueryLanguages(); if it is not
+     * then an <code>InvalidQueryException</code> is thrown.
+     *
+     * @param session the session of the current user creating the query object.
+     * @param itemMgr the item manager of the current user.
+     * @param statement the query statement.
+     * @param language the syntax of the query statement.
+     * @throws InvalidQueryException if statement is invalid or language is unsupported.
+     * @return A <code>Query</code> object.
+     */
+    public Query createQuery(SessionImpl session,
+                             ItemManager itemMgr,
+                             String statement,
+                             String language) throws InvalidQueryException;
+    
+    /**
+     * Retrieves an existing persistent query. If <code>node</code>
+     * is not a valid persisted query (that is, a node of type
+     * <code>nt:query</code>), an <code>InvalidQueryException</code>
+     * is thrown.
+     *
+     * @param absPath path to a persisted query (that is, a node of type
+     *   <code>nt:query</code>).
+     * @throws InvalidQueryException If <code>absPath</code> is not a valid persisted query
+     *   (that is, a node of type <code>nt:query</code>).
+     * @throws RepositoryException if another error occurs
+     * @return a <code>Query</code> object.
+     */
+    public Query createQuery(SessionImpl session,
+                             ItemManager itemMgr,
+                             String absPath) throws ItemNotFoundException, RepositoryException;
+}

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/QueryManagerImpl.java	Tue Jan 11 07:17:21 2005
@@ -19,7 +19,6 @@
 import org.apache.jackrabbit.core.ItemManager;
 import org.apache.jackrabbit.core.SearchManager;
 import org.apache.jackrabbit.core.SessionImpl;
-import org.apache.jackrabbit.core.search.lucene.QueryImpl;
 
 import javax.jcr.RepositoryException;
 import javax.jcr.query.InvalidQueryException;
@@ -84,7 +83,7 @@
     public Query createQuery(String statement, String language)
             throws InvalidQueryException, RepositoryException {
 
-        return new QueryImpl(session, itemMgr, searchMgr, statement, language);
+        return searchMgr.createQuery(session, itemMgr, statement, language);
     }
 
     /**
@@ -92,7 +91,8 @@
      */
     public Query getQuery(String absPath)
             throws InvalidQueryException, RepositoryException {
-        return new QueryImpl(session, itemMgr, searchMgr, absPath);
+
+        return searchMgr.createQuery(session, itemMgr, absPath);
     }
 
     /**

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/FieldNames.java	Tue Jan 11 07:17:21 2005
@@ -20,7 +20,7 @@
  * Defines field names that are used internally to store UUID, etc in the
  * search index.
  */
-public class FieldNames {
+class FieldNames {
 
     /**
      * Name of the field that contains the UUID of the node. Terms are stored

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/LuceneQueryBuilder.java	Tue Jan 11 07:17:21 2005
@@ -44,7 +44,7 @@
 
 /**
  */
-public class LuceneQueryBuilder implements QueryNodeVisitor {
+class LuceneQueryBuilder implements QueryNodeVisitor {
 
     private static final Logger log = Logger.getLogger(LuceneQueryBuilder.class);
 

Added: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NamespaceMappings.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NamespaceMappings.java?view=auto&rev=124906
==============================================================================
--- (empty file)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NamespaceMappings.java	Tue Jan 11 07:17:21 2005
@@ -0,0 +1,217 @@
+/*
+ * Copyright 2004-2005 The Apache Software Foundation or its licensors,
+ *                     as applicable.
+ *
+ * Licensed 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.core.search.lucene;
+
+import org.apache.jackrabbit.core.NamespaceResolver;
+import org.apache.jackrabbit.core.NoPrefixDeclaredException;
+import org.apache.jackrabbit.core.QName;
+import org.apache.jackrabbit.core.IllegalNameException;
+import org.apache.jackrabbit.core.UnknownPrefixException;
+import org.apache.jackrabbit.core.fs.FileSystemException;
+import org.apache.jackrabbit.core.fs.FileSystemResource;
+import org.apache.log4j.Logger;
+
+import javax.jcr.NamespaceException;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * The class <code>NamespaceMappings</code> implements a {@link
+ * org.apache.jackrabbit.core.NamespaceResolver} that holds a namespace
+ * mapping that is used internally in the search index. Storing paths with the
+ * full uri of a namespace would require too much space in the search index.
+ * <p/>
+ * Whenever a yet unknown namespace uri to prefix mapping is requested, a new
+ * prefix is created on the fly and associated with the namespace. Known
+ * namespace mappings are stored in a properties file.
+ */
+class NamespaceMappings implements NamespaceResolver {
+
+    /**
+     * Default logger instance for this class
+     */
+    private static Logger log = Logger.getLogger(NamespaceMappings.class);
+
+    /**
+     * Location of the file that persists the uri / prefix mappings
+     */
+    private final FileSystemResource storage;
+
+    /**
+     * Map of uris indexed by prefixes
+     */
+    private Map prefixToURI = new HashMap();
+
+    /**
+     * Map of prefixes indexed by uris
+     */
+    private Map uriToPrefix = new HashMap();
+
+    /**
+     * Current prefix count.
+     */
+    private int prefixCount;
+
+    /**
+     * Creates <code>NamespaceMappings</code> instance. Initial mappings are
+     * loaded from <code>file</code>.
+     *
+     * @param file the <code>File</code> to load initial mappings.
+     * @throws IOException if an error occurs while reading initial namespace
+     *                     mappings from <code>file</code>.
+     */
+    public NamespaceMappings(FileSystemResource file) throws IOException {
+        storage = file;
+        load();
+    }
+
+    /**
+     * Returns a namespace uri for a <code>prefix</code>.
+     *
+     * @param prefix the namespace prefix.
+     * @return the namespace uri.
+     * @throws NamespaceException if no namespace uri is registered for
+     *                            <code>prefix</code>.
+     */
+    public synchronized String getURI(String prefix) throws NamespaceException {
+        if (!prefixToURI.containsKey(prefix)) {
+            throw new NamespaceException(prefix + ": is not a registered namespace prefix.");
+        }
+        return (String) prefixToURI.get(prefix);
+    }
+
+    /**
+     * Returns a prefix for the namespace <code>uri</code>. If a namespace
+     * mapping exists, the already known prefix is returned; otherwise a new
+     * prefix is created and assigned to the namespace uri.
+     *
+     * @param uri the namespace uri.
+     * @return the prefix for the namespace uri.
+     * @throws NamespaceException if an yet unknown namespace uri / prefix
+     *                            mapping could not be stored.
+     */
+    public synchronized String getPrefix(String uri) throws NamespaceException {
+        String prefix = (String) uriToPrefix.get(uri);
+        if (prefix == null) {
+            // make sure prefix is not taken
+            while (prefixToURI.get(String.valueOf(prefixCount)) != null) {
+                prefixCount++;
+            }
+            prefix = String.valueOf(prefixCount);
+            prefixToURI.put(prefix, uri);
+            uriToPrefix.put(uri, prefix);
+            log.debug("adding new namespace mapping: " + prefix + " -> " + uri);
+            try {
+                store();
+            } catch (IOException e) {
+                throw new NamespaceException("Could not obtain a prefix for uri: " + uri, e);
+            }
+        }
+        return prefix;
+    }
+
+    /**
+     * Translates a property name from a session local namespace mapping
+     * into a search index private namespace mapping.
+     *
+     * @param name     the property name to translate
+     * @param resolver the <code>NamespaceResolver</code> of the local session.
+     * @return the translated property name
+     */
+    public String translatePropertyName(String name, NamespaceResolver resolver)
+            throws IllegalNameException, UnknownPrefixException {
+        QName qName = QName.fromJCRName(name, resolver);
+        try {
+            return qName.toJCRName(this);
+        } catch (NoPrefixDeclaredException e) {
+            // should never happen actually, because we create yet unknown
+            // uri mappings on the fly.
+            throw new IllegalNameException("Internal error.", e);
+        }
+    }
+
+    //-----------------------< internal >---------------------------------------
+
+    /**
+     * Loads currently known mappings from a .properties file.
+     *
+     * @throws IOException if an error occurs while reading from the file.
+     */
+    private void load() throws IOException {
+        try {
+            if (storage.exists()) {
+                InputStream in = storage.getInputStream();
+                try {
+                    Properties props = new Properties();
+                    log.debug("loading namespace mappings...");
+                    props.load(in);
+
+                    // read mappings from properties
+                    Iterator iter = props.keySet().iterator();
+                    while (iter.hasNext()) {
+                        String prefix = (String) iter.next();
+                        String uri = props.getProperty(prefix);
+                        log.debug(prefix + " -> " + uri);
+                        prefixToURI.put(prefix, uri);
+                        uriToPrefix.put(uri, prefix);
+                    }
+                    prefixCount = props.size();
+                    log.debug("namespace mappings loaded.");
+                } finally {
+                    in.close();
+                }
+            }
+        } catch (FileSystemException e) {
+            throw new IOException(e.getMessage());
+        }
+    }
+
+    /**
+     * Writes the currently known mappings into a .properties file.
+     *
+     * @throws IOException if an error occurs while writing the file.
+     */
+    private void store() throws IOException {
+        Properties props = new Properties();
+
+        // store mappings in properties
+        Iterator iter = prefixToURI.keySet().iterator();
+        while (iter.hasNext()) {
+            String prefix = (String) iter.next();
+            String uri = (String) prefixToURI.get(prefix);
+            props.setProperty(prefix, uri);
+        }
+
+        OutputStream out = null;
+        try {
+            storage.makeParentDirs();
+            out = new BufferedOutputStream(storage.getOutputStream());
+            props.store(out, null);
+        } catch (FileSystemException e) {
+            throw new IOException(e.getMessage());
+        } finally {
+            // make sure stream is closed
+            out.close();
+        }
+    }
+}

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/NodeIndexer.java	Tue Jan 11 07:17:21 2005
@@ -16,49 +16,85 @@
  */
 package org.apache.jackrabbit.core.search.lucene;
 
-import org.apache.jackrabbit.core.*;
-import org.apache.jackrabbit.core.search.NamespaceMappings;
-import org.apache.jackrabbit.core.state.*;
 import org.apache.jackrabbit.core.util.uuid.UUID;
+import org.apache.jackrabbit.core.state.NodeState;
+import org.apache.jackrabbit.core.state.ItemStateProvider;
+import org.apache.jackrabbit.core.state.NoSuchItemStateException;
+import org.apache.jackrabbit.core.state.ItemStateException;
+import org.apache.jackrabbit.core.state.PropertyState;
+import org.apache.jackrabbit.core.NodeId;
+import org.apache.jackrabbit.core.NoPrefixDeclaredException;
+import org.apache.jackrabbit.core.PropertyId;
+import org.apache.jackrabbit.core.InternalValue;
+import org.apache.jackrabbit.core.Path;
+import org.apache.jackrabbit.core.QName;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 
 import javax.jcr.NamespaceException;
-import javax.jcr.PathNotFoundException;
 import javax.jcr.PropertyType;
+import javax.jcr.RepositoryException;
 import java.util.Calendar;
 import java.util.Iterator;
 import java.util.List;
 
 /**
+ * Creates a lucene <code>Document</code> object from a {@link javax.jcr.Node}.
  *
+ * @todo add support for indexing of nt:resource. e.g. when mime type is text/*
  */
-public class NodeIndexer {
+class NodeIndexer {
 
+    /** The <code>NodeState</code> of the node to index */
     private final NodeState node;
+
+    /** The persistent item state provider */
     private final ItemStateProvider stateProvider;
-    private final Path path;
+
+    /**
+     * Namespace mappings to use for indexing. This is the internal
+     * namespace mapping.
+     */
     private final NamespaceMappings mappings;
 
+    /**
+     * Creates a new node indexer.
+     * @param node the node state to index.
+     * @param stateProvider the persistent item state manager to retrieve properties.
+     * @param mappings internal namespace mappings.
+     */
     private NodeIndexer(NodeState node,
-                        ItemStateProvider stateMgr,
-                        Path path,
+                        ItemStateProvider stateProvider,
                         NamespaceMappings mappings) {
         this.node = node;
-        this.stateProvider = stateMgr;
-        this.path = path;
+        this.stateProvider = stateProvider;
         this.mappings = mappings;
     }
 
+    /**
+     * Creates a lucene Document from a node.
+     * @param node the node state to index.
+     * @param stateProvider the state provider to retrieve property values.
+     * @param mappings internal namespace mappings.
+     * @return the lucene Document.
+     * @throws RepositoryException if an error occurs while reading property
+     *   values from the <code>ItemStateProvider</code>.
+     */
     public static Document createDocument(NodeState node,
-                                          ItemStateProvider stateMgr,
-                                          Path path,
-                                          NamespaceMappings mappings) {
-        NodeIndexer indexer = new NodeIndexer(node, stateMgr, path, mappings);
+                                          ItemStateProvider stateProvider,
+                                          NamespaceMappings mappings)
+            throws RepositoryException {
+        NodeIndexer indexer = new NodeIndexer(node, stateProvider, mappings);
         return indexer.createDoc();
     }
 
-    private Document createDoc() {
+    /**
+     * Creates a lucene Document.
+     * @return the lucene Document with the index layout.
+     * @throws RepositoryException if an error occurs while reading property
+     *   values from the <code>ItemStateProvider</code>.
+     */
+    private Document createDoc() throws RepositoryException {
         Document doc = new Document();
 
         // special fields
@@ -66,14 +102,25 @@
         doc.add(new Field(FieldNames.UUID, node.getUUID(), true, true, false));
         try {
             // parent UUID
-            if (path.denotesRoot()) {
+            if (node.getParentUUID() == null) {
+                // root node
                 doc.add(new Field(FieldNames.PARENT, "", true, true, false));
+                doc.add(new Field(FieldNames.LABEL, "", false, true, false));
             } else {
                 doc.add(new Field(FieldNames.PARENT, node.getParentUUID(), true, true, false));
+                NodeState parent = (NodeState) stateProvider.getItemState(
+                        new NodeId(node.getParentUUID()));
+                List entries = parent.getChildNodeEntries(node.getUUID());
+                for (Iterator it = entries.iterator(); it.hasNext();) {
+                    NodeState.ChildNodeEntry child = (NodeState.ChildNodeEntry) it.next();
+                    String name = child.getName().toJCRName(mappings);
+                    doc.add(new Field(FieldNames.LABEL, name, false, true, false));
+                }
             }
-            // Label
-            doc.add(new Field(FieldNames.LABEL, path.getNameElement().toJCRName(mappings),
-                    false, true, false));
+        } catch (NoSuchItemStateException e) {
+            throw new RepositoryException("Error while indexing node: " + node.getUUID(), e);
+        } catch (ItemStateException e) {
+            throw new RepositoryException("Error while indexing node: " + node.getUUID(), e);
         } catch (NoPrefixDeclaredException e) {
             // will never happen, because this.mappings will dynamically add
             // unknown uri<->prefix mappings
@@ -90,14 +137,20 @@
                     addValue(doc, values[i], propState.getName());
                 }
             } catch (NoSuchItemStateException e) {
-                // FIXME do logging? throw?
+                throw new RepositoryException("Error while indexing node: " + node.getUUID(), e);
             } catch (ItemStateException e) {
-                // FIXME do logging? throw?
+                throw new RepositoryException("Error while indexing node: " + node.getUUID(), e);
             }
         }
         return doc;
     }
 
+    /**
+     * Adds a value to the lucene Document.
+     * @param doc the document.
+     * @param value the internal jackrabbit value.
+     * @param name the name of the property.
+     */
     private void addValue(Document doc, InternalValue value, QName name) {
         String fieldName = name.toString();
         try {

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/PersistentIndex.java	Tue Jan 11 07:17:21 2005
@@ -27,7 +27,7 @@
  * Implements a lucene index which is based on a
  * {@link org.apache.jackrabbit.core.fs.FileSystem}.
  */
-public class PersistentIndex extends AbstractIndex {
+class PersistentIndex extends AbstractIndex {
 
     private final FileSystem fs;
 

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryImpl.java	Tue Jan 11 07:17:21 2005
@@ -16,22 +16,42 @@
  */
 package org.apache.jackrabbit.core.search.lucene;
 
-import org.apache.jackrabbit.core.*;
 import org.apache.jackrabbit.core.nodetype.NodeTypeRegistry;
 import org.apache.jackrabbit.core.search.QueryParser;
 import org.apache.jackrabbit.core.search.QueryRootNode;
+import org.apache.jackrabbit.core.search.OrderQueryNode;
+import org.apache.jackrabbit.core.QName;
+import org.apache.jackrabbit.core.NamespaceRegistryImpl;
+import org.apache.jackrabbit.core.SessionImpl;
+import org.apache.jackrabbit.core.ItemManager;
+import org.apache.jackrabbit.core.Path;
+import org.apache.jackrabbit.core.MalformedPathException;
+import org.apache.jackrabbit.core.NoPrefixDeclaredException;
+import org.apache.jackrabbit.core.AccessManagerImpl;
+import org.apache.jackrabbit.core.NodeId;
+import org.apache.jackrabbit.core.AccessManager;
+import org.apache.jackrabbit.core.NamespaceResolver;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.Query;
 
-import javax.jcr.*;
 import javax.jcr.nodetype.ConstraintViolationException;
 import javax.jcr.nodetype.NodeType;
 import javax.jcr.nodetype.NodeTypeManager;
 import javax.jcr.query.InvalidQueryException;
-import javax.jcr.query.Query;
 import javax.jcr.query.QueryResult;
+import javax.jcr.ItemNotFoundException;
+import javax.jcr.RepositoryException;
+import javax.jcr.Node;
+import javax.jcr.ItemExistsException;
+import javax.jcr.PathNotFoundException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.io.IOException;
 
 /**
  */
-public class QueryImpl implements Query {
+class QueryImpl implements javax.jcr.query.Query {
 
     /**
      * jcr:statement
@@ -51,22 +71,22 @@
 
     private final ItemManager itemMgr;
 
+    private final SearchIndex index;
+
     private final String statement;
 
     private final String language;
 
-    private final SearchManager searchMgr;
-
     private Path path;
 
     public QueryImpl(SessionImpl session,
                      ItemManager itemMgr,
-                     SearchManager searchMgr,
+                     SearchIndex index,
                      String statement,
                      String language) throws InvalidQueryException {
         this.session = session;
         this.itemMgr = itemMgr;
-        this.searchMgr = searchMgr;
+        this.index = index;
         this.statement = statement;
         this.language = language;
 
@@ -77,13 +97,13 @@
 
     public QueryImpl(SessionImpl session,
                      ItemManager itemMgr,
-                     SearchManager searchMgr,
+                     SearchIndex index,
                      String absPath)
             throws ItemNotFoundException, InvalidQueryException, RepositoryException {
 
         this.session = session;
         this.itemMgr = itemMgr;
-        this.searchMgr = searchMgr;
+        this.index = index;
 
         try {
             Node query = null;
@@ -117,7 +137,47 @@
     }
 
     public QueryResult execute() throws RepositoryException {
-        return searchMgr.execute(itemMgr, root, session);
+        // build lucene query
+        Query query = LuceneQueryBuilder.createQuery(root,
+                session, index.getNamespaceMappings(), index.getAnalyzer());
+
+        OrderQueryNode orderNode = root.getOrderNode();
+        // FIXME according to spec this should be descending
+        // by default. this contrasts to standard sql semantics
+        // where default is ascending.
+        boolean[] orderSpecs = null;
+        String[] orderProperties = null;
+        if (orderNode != null) {
+            orderProperties = orderNode.getOrderByProperties();
+            orderSpecs = orderNode.getOrderBySpecs();
+        } else {
+            orderProperties = new String[0];
+            orderSpecs = new boolean[0];
+        }
+
+
+        List uuids;
+        AccessManagerImpl accessMgr = session.getAccessManager();
+
+        // execute it
+        try {
+            Hits result = index.executeQuery(query, orderProperties, orderSpecs);
+            uuids = new ArrayList(result.length());
+            for (int i = 0; i < result.length(); i++) {
+                String uuid = result.doc(i).get(FieldNames.UUID);
+                // check access
+                if (accessMgr.isGranted(new NodeId(uuid), AccessManager.READ)) {
+                    uuids.add(uuid);
+                }
+            }
+        } catch (IOException e) {
+            uuids = Collections.EMPTY_LIST;
+        }
+
+        // return QueryResult
+        return new QueryResultImpl(itemMgr,
+                (String[]) uuids.toArray(new String[uuids.size()]),
+                root.getSelectProperties());
     }
 
     public String getStatement() {

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/QueryResultImpl.java	Tue Jan 11 07:17:21 2005
@@ -27,7 +27,7 @@
 /**
  * Implements the <code>javax.jcr.query.QueryResult</code> interface.
  */
-public class QueryResultImpl implements QueryResult {
+class QueryResultImpl implements QueryResult {
 
     private static final Logger log = Logger.getLogger(QueryResultImpl.class);
 

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/SearchIndex.java	Tue Jan 11 07:17:21 2005
@@ -17,27 +17,40 @@
 package org.apache.jackrabbit.core.search.lucene;
 
 import EDU.oswego.cs.dl.util.concurrent.FIFOReadWriteLock;
-import org.apache.jackrabbit.core.fs.FileSystem;
 import org.apache.jackrabbit.core.fs.FileSystemException;
+import org.apache.jackrabbit.core.fs.FileSystemResource;
+import org.apache.jackrabbit.core.search.AbstractQueryHandler;
+import org.apache.jackrabbit.core.state.NodeState;
+import org.apache.jackrabbit.core.SessionImpl;
+import org.apache.jackrabbit.core.ItemManager;
 import org.apache.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Hits;
 import org.apache.lucene.search.Query;
 
+import javax.jcr.query.InvalidQueryException;
+import javax.jcr.RepositoryException;
+import javax.jcr.ItemNotFoundException;
 import java.io.IOException;
 
 /**
+ * Implements a {@link org.apache.jackrabbit.core.search.QueryHandler} using
+ * Lucene.
  */
-public class SearchIndex {
+public class SearchIndex extends AbstractQueryHandler {
 
     private static final Logger log = Logger.getLogger(SearchIndex.class);
 
+    /** Name of the file to persist search internal namespace mappings */
+    private static final String NS_MAPPING_FILE = "ns_mappings.properties";
+
     /**
      * 512k default size
      */
-    //private static final long DEFAULT_MERGE_SIZE = 512 * 1024 * 1024;
+    //private static final long DEFAULT_MERGE_SIZE = 512 * 1024;
 
     //private long mergeSize = DEFAULT_MERGE_SIZE;
 
@@ -47,23 +60,43 @@
 
     private final Analyzer analyzer;
 
+    private NamespaceMappings nsMappings;
+
     private final FIFOReadWriteLock readWriteLock = new FIFOReadWriteLock();
 
-    public SearchIndex(FileSystem fs, Analyzer analyzer)
-            throws IOException {
+    /**
+     * Default constructor.
+     */
+    public SearchIndex() {
+        this.analyzer = new StandardAnalyzer();
         //volatileIndex = new VolatileIndex(analyzer);
+    }
+
+    /**
+     * Initializes this <code>QueryHandler</code>.
+     * @throws IOException if an error occurs while initializing this handler.
+     */
+    protected void doInit() throws IOException {
         boolean create;
         try {
-            create = !fs.exists("segments");
-            persistentIndex = new PersistentIndex(fs, create, analyzer);
+            create = !getFileSystem().exists("segments");
+            persistentIndex = new PersistentIndex(getFileSystem(), create, analyzer);
             persistentIndex.setUseCompoundFile(true);
-            this.analyzer = analyzer;
+            FileSystemResource mapFile = new FileSystemResource(getFileSystem(), NS_MAPPING_FILE);
+            nsMappings = new NamespaceMappings(mapFile);
         } catch (FileSystemException e) {
             throw new IOException(e.getMessage());
         }
     }
 
-    public void addDocument(Document doc) throws IOException {
+    /**
+     * Adds the <code>node</code> to the search index.
+     * @param node the node to add.
+     * @throws RepositoryException if an error occurs while indexing the node.
+     * @throws IOException if an error occurs while adding the node to the index.
+     */
+    public void addNode(NodeState node) throws RepositoryException, IOException {
+        Document doc = NodeIndexer.createDocument(node, getItemStateProvider(), nsMappings);
         try {
             readWriteLock.writeLock().acquire();
         } catch (InterruptedException e) {
@@ -87,7 +120,14 @@
         */
     }
 
-    public void removeDocument(Term idTerm) throws IOException {
+    /**
+     * Removes the node with <code>uuid</code> from the search index.
+     * @param uuid the UUID of the node to remove from the index.
+     * @throws IOException if an error occurs while removing the node from
+     * the index.
+     */
+    public void deleteNode(String uuid) throws IOException {
+        Term idTerm = new Term(FieldNames.UUID, uuid);
         try {
             readWriteLock.writeLock().acquire();
         } catch (InterruptedException e) {
@@ -104,6 +144,53 @@
         //volatileIndex.removeDocument(idTerm);
     }
 
+    /**
+     * Creates a new query by specifying the query statement itself and the
+     * language in which the query is stated.  If the query statement is
+     * syntactically invalid, given the language specified, an
+     * InvalidQueryException is thrown. <code>language</code> must specify a query language
+     * string from among those returned by QueryManager.getSupportedQueryLanguages(); if it is not
+     * then an <code>InvalidQueryException</code> is thrown.
+     *
+     * @param session the session of the current user creating the query object.
+     * @param itemMgr the item manager of the current user.
+     * @param statement the query statement.
+     * @param language the syntax of the query statement.
+     * @throws InvalidQueryException if statement is invalid or language is unsupported.
+     * @return A <code>Query</code> object.
+     */
+    public javax.jcr.query.Query createQuery(SessionImpl session,
+                                             ItemManager itemMgr,
+                                             String statement,
+                                             String language)
+            throws InvalidQueryException {
+        return new QueryImpl(session, itemMgr, this, statement, language);
+    }
+
+    /**
+     * Retrieves an existing persistent query. If <code>node</code>
+     * is not a valid persisted query (that is, a node of type
+     * <code>nt:query</code>), an <code>InvalidQueryException</code>
+     * is thrown.
+     *
+     * @param absPath path to a persisted query (that is, a node of type
+     *   <code>nt:query</code>).
+     * @throws InvalidQueryException If <code>absPath</code> is not a valid persisted query
+     *   (that is, a node of type <code>nt:query</code>).
+     * @throws RepositoryException if another error occurs
+     * @return a <code>Query</code> object.
+     */
+    public javax.jcr.query.Query createQuery(SessionImpl session,
+                                             ItemManager itemMgr,
+                                             String absPath)
+            throws ItemNotFoundException, RepositoryException {
+        return new QueryImpl(session, itemMgr, this, absPath);
+    }
+
+    /**
+     * Closes this <code>QueryHandler</code> and frees resources attached
+     * to this handler.
+     */
     public void close() {
         /*
         try {
@@ -117,7 +204,7 @@
         persistentIndex.close();
     }
 
-    public Hits executeQuery(Query query,
+    Hits executeQuery(Query query,
                              String[] orderProps,
                              boolean[] orderSpecs) throws IOException {
         try {
@@ -144,8 +231,12 @@
         return hits;
     }
 
-    public Analyzer getAnalyzer() {
+    Analyzer getAnalyzer() {
         return analyzer;
+    }
+
+    NamespaceMappings getNamespaceMappings() {
+        return nsMappings;
     }
 
     //--------------------------< properties >----------------------------------

Modified: incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java
Url: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java?view=diff&rev=124906&p1=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java&r1=124905&p2=incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java&r2=124906
==============================================================================
--- incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java	(original)
+++ incubator/jackrabbit/trunk/src/java/org/apache/jackrabbit/core/search/lucene/VolatileIndex.java	Tue Jan 11 07:17:21 2005
@@ -24,7 +24,7 @@
 
 /**
  */
-public class VolatileIndex extends AbstractIndex {
+class VolatileIndex extends AbstractIndex {
 
     VolatileIndex(Analyzer analyzer) throws IOException {
         super(analyzer, new RAMDirectory());