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/08/15 14:12:14 UTC

svn commit: r232801 [1/2] - in /incubator/jackrabbit/trunk/core: applications/test/workspaces/default/ applications/test/workspaces/test/ src/java/org/apache/jackrabbit/core/ src/java/org/apache/jackrabbit/core/query/ src/java/org/apache/jackrabbit/cor...

Author: mreutegg
Date: Mon Aug 15 05:11:52 2005
New Revision: 232801

URL: http://svn.apache.org/viewcvs?rev=232801&view=rev
Log:
JCR-178: Allow concurrent index updates and queries

Added:
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java   (with props)
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java   (with props)
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java   (with props)
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java   (with props)
    incubator/jackrabbit/trunk/core/src/test/org/apache/jackrabbit/core/query/ConcurrentQueryTest.java   (with props)
Modified:
    incubator/jackrabbit/trunk/core/applications/test/workspaces/default/workspace.xml
    incubator/jackrabbit/trunk/core/applications/test/workspaces/test/workspace.xml
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/SearchManager.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/AbstractQueryHandler.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/QueryHandler.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/AbstractIndex.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CachingMultiReader.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ConsistencyCheck.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/DescendantSelfAxisQuery.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/IndexInfos.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/MultiIndex.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/PersistentIndex.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryImpl.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/RedoLog.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SearchIndex.java
    incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/VolatileIndex.java
    incubator/jackrabbit/trunk/core/src/test/org/apache/jackrabbit/core/query/TestAll.java

Modified: incubator/jackrabbit/trunk/core/applications/test/workspaces/default/workspace.xml
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/applications/test/workspaces/default/workspace.xml?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/applications/test/workspaces/default/workspace.xml (original)
+++ incubator/jackrabbit/trunk/core/applications/test/workspaces/default/workspace.xml Mon Aug 15 05:11:52 2005
@@ -20,7 +20,7 @@
   -->
   <SearchIndex class="org.apache.jackrabbit.core.query.lucene.SearchIndex">
     <param name="useCompoundFile" value="true" />
-    <param name="minMergeDocs" value="1000" />
+    <param name="minMergeDocs" value="100" />
     <param name="maxMergeDocs" value="100000" />
     <param name="mergeFactor" value="10" />
     <param name="bufferSize" value="10"/>

Modified: incubator/jackrabbit/trunk/core/applications/test/workspaces/test/workspace.xml
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/applications/test/workspaces/test/workspace.xml?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/applications/test/workspaces/test/workspace.xml (original)
+++ incubator/jackrabbit/trunk/core/applications/test/workspaces/test/workspace.xml Mon Aug 15 05:11:52 2005
@@ -20,7 +20,7 @@
   -->
   <SearchIndex class="org.apache.jackrabbit.core.query.lucene.SearchIndex">
     <param name="useCompoundFile" value="true" />
-    <param name="minMergeDocs" value="1000" />
+    <param name="minMergeDocs" value="100" />
     <param name="maxMergeDocs" value="100000" />
     <param name="mergeFactor" value="10" />
     <param name="bufferSize" value="10"/>

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/SearchManager.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/SearchManager.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/SearchManager.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/SearchManager.java Mon Aug 15 05:11:52 2005
@@ -17,6 +17,7 @@
 package org.apache.jackrabbit.core;
 
 import org.apache.commons.collections.BeanMap;
+import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
 import org.apache.jackrabbit.core.config.SearchConfig;
 import org.apache.jackrabbit.core.fs.FileSystem;
 import org.apache.jackrabbit.core.fs.FileSystemException;
@@ -29,6 +30,7 @@
 import org.apache.jackrabbit.core.state.ItemStateException;
 import org.apache.jackrabbit.core.state.ItemStateManager;
 import org.apache.jackrabbit.core.state.NodeState;
+import org.apache.jackrabbit.core.state.ItemState;
 import org.apache.jackrabbit.name.Path;
 import org.apache.log4j.Logger;
 
@@ -135,28 +137,6 @@
     }
 
     /**
-     * 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 {
-        handler.addNode(node);
-    }
-
-    /**
-     * 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 {
-        handler.deleteNode(uuid);
-    }
-
-    /**
      * Closes this <code>SearchManager</code> and also closes the
      * {@link FileSystem} configured in {@link SearchConfig}.
      */
@@ -259,24 +239,26 @@
             }
         }
 
-        for (Iterator it = removedNodes.iterator(); it.hasNext();) {
-            try {
-                deleteNode((String) it.next());
-            } catch (IOException e) {
-                log.error("Error deleting node from index.", e);
-            }
-        }
-        for (Iterator it = addedNodes.iterator(); it.hasNext();) {
-            try {
-                addNode((NodeState) itemMgr.getItemState(new NodeId((String) it.next())));
-            } catch (ItemStateException e) {
-                log.error("Error indexing node.", e);
-            } catch (RepositoryException e) {
-                log.error("Error indexing node.", e);
-            } catch (IOException e) {
-                log.error("Error indexing node.", e);
+        Iterator addedStates = new AbstractIteratorDecorator(addedNodes.iterator()) {
+            public Object next() {
+                ItemState item = null;
+                String uuid = (String) super.next();
+                try {
+                    item = itemMgr.getItemState(new NodeId(uuid));
+                } catch (ItemStateException e) {
+                    log.error("Unable to index node " + uuid + ": does not exist");
+                }
+                return item;
             }
+        };
+        try {
+            handler.updateNodes(removedNodes.iterator(), addedStates);
+        } catch (RepositoryException e) {
+            log.error("Error indexing node.", e);
+        } catch (IOException e) {
+            log.error("Error indexing node.", e);
         }
+
         if (log.isDebugEnabled()) {
             log.debug("onEvent: indexing finished in "
                     + String.valueOf(System.currentTimeMillis() - time)

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/AbstractQueryHandler.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/AbstractQueryHandler.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/AbstractQueryHandler.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/AbstractQueryHandler.java Mon Aug 15 05:11:52 2005
@@ -16,7 +16,11 @@
  */
 package org.apache.jackrabbit.core.query;
 
+import org.apache.jackrabbit.core.state.NodeState;
+
+import javax.jcr.RepositoryException;
 import java.io.IOException;
+import java.util.Iterator;
 
 /**
  * Implements default behaviour for some methods of {@link QueryHandler}.
@@ -53,5 +57,26 @@
      */
     public QueryHandlerContext getContext() {
         return context;
+    }
+
+    /**
+     * This default implementation calls the individual {@link #deleteNode(String)}
+     * and {@link #addNode(org.apache.jackrabbit.core.state.NodeState)} methods
+     * for each entry in the iterators. First the nodes to remove are processed
+     * then the nodes to add.
+     *
+     * @param remove uuids of nodes to remove.
+     * @param add NodeStates to add.
+     * @throws RepositoryException if an error occurs while indexing a node.
+     * @throws IOException if an error occurs while updating the index.
+     */
+    public synchronized void updateNodes(Iterator remove, Iterator add)
+            throws RepositoryException, IOException {
+        while (remove.hasNext()) {
+            deleteNode((String) remove.next());
+        }
+        while (add.hasNext()) {
+            addNode((NodeState) add.next());
+        }
     }
 }

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/QueryHandler.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/QueryHandler.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/QueryHandler.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/QueryHandler.java Mon Aug 15 05:11:52 2005
@@ -23,6 +23,7 @@
 import javax.jcr.RepositoryException;
 import javax.jcr.query.InvalidQueryException;
 import java.io.IOException;
+import java.util.Iterator;
 
 /**
  * Defines an interface for the actual node indexing and query execution.
@@ -55,6 +56,20 @@
      * @throws IOException if an error occurs while deleting the node.
      */
     void deleteNode(String uuid) throws IOException;
+
+    /**
+     * Updates the index in an atomic operation. Some nodes may be removed and
+     * added again in the same updateNodes() call, which is equivalent to an
+     * node update.
+     *
+     * @param remove Iterator of uuid <code>String</code>s of nodes to delete
+     * @param add    Iterator of <code>NodeState</code> instance to add to the
+     *               index.
+     * @throws RepositoryException if an error occurs while indexing a node.
+     * @throws IOException if an error occurs while updating the index.
+     */
+    void updateNodes(Iterator remove, Iterator add)
+            throws RepositoryException, IOException;
 
     /**
      * Closes this <code>QueryHandler</code> and frees resources attached

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/AbstractIndex.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/AbstractIndex.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/AbstractIndex.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/AbstractIndex.java Mon Aug 15 05:11:52 2005
@@ -27,9 +27,21 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintStream;
+import java.util.BitSet;
 
 /**
  * Implements common functionality for a lucene index.
+ * <p/>
+ * Note on synchronization: This class is not entirely thread-safe. Certain
+ * concurrent access is however allowed. Read-only access on this index using
+ * {@link #getReadOnlyIndexReader()} is thread-safe. That is, multiple threads
+ * my call that method concurrently and use the returned IndexReader at the same
+ * time.<br/>
+ * Modifying threads must be synchronized externally in a way that only one
+ * thread is using the returned IndexReader and IndexWriter instances returned
+ * by {@link #getIndexReader()} and {@link #getIndexWriter()} at a time.<br/>
+ * Concurrent access by <b>one</b> modifying thread and multiple read-only
+ * threads is safe!
  */
 abstract class AbstractIndex {
 
@@ -42,8 +54,8 @@
     /** The currently set IndexWriter or <code>null</code> if none is set */
     private IndexWriter indexWriter;
 
-    /** The currently set IndexReader of <code>null</code> if none is set */
-    private IndexReader indexReader;
+    /** The currently set IndexReader or <code>null</code> if none is set */
+    private CommittableIndexReader indexReader;
 
     /** The underlying Directory where the index is stored */
     private Directory directory;
@@ -63,6 +75,9 @@
     /** mergeFactor config parameter */
     private int mergeFactor = 10;
 
+    /** The shared IndexReader for all read-only IndexReaders */
+    private SharedIndexReader sharedReader;
+
     /**
      * Constructs an index with an <code>analyzer</code> and a
      * <code>directory</code>.
@@ -94,16 +109,21 @@
     }
 
     /**
-     * Adds a document to this index.
+     * Adds a document to this index and invalidates the shared reader.
+     *
      * @param doc the document to add.
      * @throws IOException if an error occurs while writing to the index.
      */
     void addDocument(Document doc) throws IOException {
         getIndexWriter().addDocument(doc);
+        invalidateSharedReader();
     }
 
     /**
-     * Removes the document from this index.
+     * Removes the document from this index. This call will not invalidate
+     * the shared reader. If a subclass whishes to do so, it should overwrite
+     * this method and call {@link #invalidateSharedReader()}.
+     *
      * @param idTerm the id term of the document to remove.
      * @throws IOException if an error occurs while removing the document.
      * @return number of documents deleted
@@ -113,7 +133,9 @@
     }
 
     /**
-     * Returns an <code>IndexReader</code> on this index.
+     * Returns an <code>IndexReader</code> on this index. This index reader
+     * may be used to delete documents.
+     *
      * @return an <code>IndexReader</code> on this index.
      * @throws IOException if the reader cannot be obtained.
      */
@@ -124,15 +146,40 @@
             indexWriter = null;
         }
         if (indexReader == null) {
-            indexReader = IndexReader.open(getDirectory());
-            if (useCachingReader()) {
-                indexReader = new CachingIndexReader(indexReader);
-            }
+            indexReader = new CommittableIndexReader(IndexReader.open(getDirectory()));
         }
         return indexReader;
     }
 
     /**
+     * Returns a read-only index reader, that can be used concurrently with
+     * other threads writing to this index. The returned index reader is
+     * read-only, that is, any attempt to delete a document from the index
+     * will throw an <code>UnsupportedOperationException</code>.
+     *
+     * @return a read-only index reader.
+     * @throws IOException if an error occurs while obtaining the index reader.
+     */
+    protected synchronized ReadOnlyIndexReader getReadOnlyIndexReader()
+            throws IOException {
+        // get current modifiable index reader
+        IndexReader modifiableReader = getIndexReader();
+        // capture snapshot of deleted documents
+        BitSet deleted = new BitSet(modifiableReader.maxDoc());
+        for (int i = 0; i < modifiableReader.maxDoc(); i++) {
+            if (modifiableReader.isDeleted(i)) {
+                deleted.set(i);
+            }
+        }
+        if (sharedReader == null) {
+            // create new shared reader
+            sharedReader = new SharedIndexReader(new CachingIndexReader(IndexReader.open(getDirectory())));
+        }
+        return new ReadOnlyIndexReader(sharedReader, deleted);
+    }
+
+
+    /**
      * Returns an <code>IndexWriter</code> on this index.
      * @return an <code>IndexWriter</code> on this index.
      * @throws IOException if the writer cannot be obtained.
@@ -165,14 +212,7 @@
         }
 
         if (indexReader != null) {
-            if (indexReader instanceof CachingIndexReader) {
-                // only commit changes, do not close
-                log.debug("committing IndexReader.");
-                ((CachingIndexReader) indexReader).commitDeleted();
-            } else {
-                indexReader.close();
-                indexReader = null;
-            }
+            indexReader.commitDeleted();
         }
         if (indexWriter != null) {
             log.debug("committing IndexWriter.");
@@ -184,7 +224,7 @@
     /**
      * Closes this index, releasing all held resources.
      */
-    void close() {
+    synchronized void close() {
         if (indexWriter != null) {
             try {
                 indexWriter.close();
@@ -201,6 +241,13 @@
             }
             indexReader = null;
         }
+        if (sharedReader != null) {
+            try {
+                sharedReader.close();
+            } catch (IOException e) {
+                log.warn("Exception closing index reader: " + e.toString());
+            }
+        }
         if (directory != null) {
             try {
                 directory.close();
@@ -211,12 +258,16 @@
     }
 
     /**
-     * Returns <code>true</code> if this index should use a
-     * {@link CachingIndexReader}, <code>false</code> otherwise.
-     * @return <code>true</code> if index reader should use caching.
+     * Closes the shared reader.
+     *
+     * @throws IOException if an error occurs while closing the reader.
      */
-    protected boolean useCachingReader() {
-        return false;
+    protected synchronized void invalidateSharedReader() throws IOException {
+        // invalidate shared reader
+        if (sharedReader != null) {
+            sharedReader.close();
+            sharedReader = null;
+        }
     }
 
     //-------------------------< properties >-----------------------------------

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CachingMultiReader.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CachingMultiReader.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CachingMultiReader.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CachingMultiReader.java Mon Aug 15 05:11:52 2005
@@ -41,12 +41,18 @@
     private int[] starts;
 
     /**
+     * Reference count. Every time close is called refCount is decremented. If
+     * refCount drops to zero the underlying readers are closed as well.
+     */
+    private int refCount = 1;
+
+    /**
      * Creates a new <code>CachingMultiReader</code> based on sub readers.
      * <p/>
      * This <code>CachingMultiReader</code> poses type requirements on the
      * <code>subReaders</code>: all but one sub readers must be a
-     * {@link CachingIndexReader}. The single allowed sub reader not of type
-     * {@link CachingIndexReader} must be the last reader in
+     * {@link ReadOnlyIndexReader}. The single allowed sub reader not of type
+     * {@link ReadOnlyIndexReader} must be the last reader in
      * <code>subReaders</code>! Otherwise this constructor will throw an
      * {@link IllegalArgumentException}.
      *
@@ -58,10 +64,10 @@
     public CachingMultiReader(IndexReader[] subReaders)
             throws IOException, IllegalArgumentException {
         super(subReaders);
-        // check readers, all but last must be a CachingIndexReader
+        // check readers, all but last must be a ReadOnlyIndexReader
         for (int i = 0; i < subReaders.length - 1; i++) {
-            if (!(subReaders[i] instanceof CachingIndexReader)) {
-                throw new IllegalArgumentException("subReader " + i + " must be of type CachingIndexReader");
+            if (!(subReaders[i] instanceof ReadOnlyIndexReader)) {
+                throw new IllegalArgumentException("subReader " + i + " must be of type ReadOnlyIndexReader");
             }
         }
         this.subReaders = subReaders;
@@ -94,6 +100,25 @@
             return new MultiTermDocs(termDocs, starts);
         }
         return super.termDocs(term);
+    }
+
+    /**
+     * Increments the reference count of this reader. Each call to this method
+     * must later be acknowledged by a call to {@link #close()}
+     */
+    synchronized void incrementRefCount() {
+        refCount++;
+    }
+
+    /**
+     * Decrements the reference count and closes the underlying readers if this
+     * reader is not in use anymore.
+     * @throws IOException if an error occurs while closing this reader.
+     */
+    protected synchronized void doClose() throws IOException {
+        if (--refCount == 0) {
+            super.doClose();
+        }
     }
 
     /**

Added: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java?rev=232801&view=auto
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java (added)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java Mon Aug 15 05:11:52 2005
@@ -0,0 +1,47 @@
+/*
+ * 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.query.lucene;
+
+import org.apache.lucene.index.FilterIndexReader;
+import org.apache.lucene.index.IndexReader;
+
+import java.io.IOException;
+
+/**
+ * Wraps an <code>IndexReader</code> and allows to commit changes without
+ * closing the reader.
+ */
+class CommittableIndexReader extends FilterIndexReader {
+
+    /**
+     * Creates a new <code>CommittableIndexReader</code> based on <code>in</code>.
+     *
+     * @param in the <code>IndexReader</code> to wrap.
+     */
+    CommittableIndexReader(IndexReader in) {
+        super(in);
+    }
+
+    /**
+     * Commits the documents marked as deleted to disc.
+     *
+     * @throws IOException if an error occurs while writing.
+     */
+    void commitDeleted() throws IOException {
+        commit();
+    }
+}

Propchange: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/CommittableIndexReader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ConsistencyCheck.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ConsistencyCheck.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ConsistencyCheck.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ConsistencyCheck.java Mon Aug 15 05:11:52 2005
@@ -119,11 +119,14 @@
                     log.warn("Not repairable: " + error);
                     notRepairable++;
                 }
-            } catch (IOException e) {
+            } catch (Exception e) {
                 if (ignoreFailure) {
                     log.warn("Exception while reparing: " + e);
                 } else {
-                    throw e;
+                    if (!(e instanceof IOException)) {
+                        e = new IOException(e.getMessage());
+                    }
+                    throw (IOException) e;
                 }
             }
         }
@@ -151,20 +154,24 @@
         // collect all documents
         documents = new HashMap();
         IndexReader reader = index.getIndexReader();
-        for (int i = 0; i < reader.maxDoc(); i++) {
-            if (reader.isDeleted(i)) {
-                continue;
-            }
-            Document d = reader.document(i);
-            String uuid = d.get(FieldNames.UUID);
-            if (stateMgr.hasItemState(new NodeId(uuid))) {
-                Document old = (Document) documents.put(uuid, d);
-                if (old != null) {
-                    multipleEntries.add(uuid);
+        try {
+            for (int i = 0; i < reader.maxDoc(); i++) {
+                if (reader.isDeleted(i)) {
+                    continue;
+                }
+                Document d = reader.document(i);
+                String uuid = d.get(FieldNames.UUID);
+                if (stateMgr.hasItemState(new NodeId(uuid))) {
+                    Document old = (Document) documents.put(uuid, d);
+                    if (old != null) {
+                        multipleEntries.add(uuid);
+                    }
+                } else {
+                    errors.add(new NodeDeleted(uuid));
                 }
-            } else {
-                errors.add(new NodeDeleted(uuid));
             }
+        } finally {
+            reader.close();
         }
 
         // create multiple entries errors

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/DescendantSelfAxisQuery.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/DescendantSelfAxisQuery.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/DescendantSelfAxisQuery.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/DescendantSelfAxisQuery.java Mon Aug 15 05:11:52 2005
@@ -26,6 +26,7 @@
 import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.document.Document;
 
 import java.io.IOException;
 import java.util.BitSet;
@@ -248,13 +249,15 @@
                 }
 
                 // check if nextDoc is a descendant of one of the context nodes
-                String parentUUID = reader.document(nextDoc).get(FieldNames.PARENT);
+                Document d = reader.document(nextDoc);
+                String parentUUID = d.get(FieldNames.PARENT);
                 while (parentUUID != null && !contextUUIDs.contains(parentUUID)) {
                     // traverse
                     TermDocs ancestor = reader.termDocs(new Term(FieldNames.UUID, parentUUID));
                     try {
                         if (ancestor.next()) {
-                            parentUUID = reader.document(ancestor.doc()).get(FieldNames.PARENT);
+                            d = reader.document(ancestor.doc());
+                            parentUUID = d.get(FieldNames.PARENT);
                             if (parentUUID.length() == 0) {
                                 parentUUID = null;
                             }
@@ -266,6 +269,9 @@
                     }
                 }
                 if (parentUUID != null) {
+                    // since current doc is a descendant of one of the context
+                    // docs we can promote uuid of doc to the context uuids
+                    contextUUIDs.add(d.get(FieldNames.UUID));
                     return true;
                 }
                 // try next

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/IndexInfos.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/IndexInfos.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/IndexInfos.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/IndexInfos.java Mon Aug 15 05:11:52 2005
@@ -45,13 +45,36 @@
     private List indexes = new ArrayList();
 
     /**
+     * Name of the file where the infos are stored.
+     */
+    private final String name;
+
+    /**
+     * Creates a new IndexInfos using <code>fileName</code>.
+     *
+     * @param fileName the name of the file where infos are stored.
+     */
+    IndexInfos(String fileName) {
+        this.name = fileName;
+    }
+
+    /**
+     * Returns the name of the file where infos are stored.
+     * 
+     * @return the name of the file where infos are stored.
+     */
+    String getFileName() {
+        return name;
+    }
+
+    /**
      * Reads the index infos.
      * @param fs the base file system
      * @throws FileSystemException if an error occurs.
      * @throws IOException if an error occurs.
      */
     void read(FileSystem fs) throws FileSystemException, IOException {
-        DataInputStream input = new DataInputStream(fs.getInputStream("indexes"));
+        DataInputStream input = new DataInputStream(fs.getInputStream(name));
         try {
             counter = input.readInt();
             for (int i = input.readInt(); i > 0; i--) {
@@ -74,7 +97,7 @@
             return;
         }
 
-        DataOutputStream output = new DataOutputStream(fs.getOutputStream("indexes.new"));
+        DataOutputStream output = new DataOutputStream(fs.getOutputStream(name + ".new"));
         try {
             output.writeInt(counter);
             output.writeInt(indexes.size());
@@ -84,7 +107,7 @@
         } finally {
             output.close();
         }
-        fs.move("indexes.new", "indexes");
+        fs.move(name + ".new", name);
         dirty = false;
     }
 

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/MultiIndex.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/MultiIndex.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/MultiIndex.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/MultiIndex.java Mon Aug 15 05:11:52 2005
@@ -29,16 +29,16 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.commons.collections.iterators.EmptyIterator;
 
 import javax.jcr.RepositoryException;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.Arrays;
 
 /**
  * A <code>MultiIndex</code> consists of a {@link VolatileIndex} and multiple
@@ -58,18 +58,6 @@
  * {@link SearchIndex#setMergeFactor(int)} and {@link SearchIndex#setMinMergeDocs(int)}. For detailed
  * description of the configuration parameters see also the lucene
  * <code>IndexWriter</code> class.
- * <p/>
- * This class is not thread-safe. Clients of this class must ensure
- * synchronization of multiple threads. The following conditions must hold true:
- * <ul>
- * <li>Only one thread may use {@link #addDocument(org.apache.lucene.document.Document)}
- * or {@link #removeDocument(org.apache.lucene.index.Term)} at a time.</li>
- * <li>While a thread uses the <code>IndexReader</code> returned by
- * {@link #getIndexReader()} other threads must not call {@link #addDocument(org.apache.lucene.document.Document)}
- * or {@link #removeDocument(org.apache.lucene.index.Term)}</li>
- * <li>Multiple threads may use the <code>IndexReader</code> returned by
- * {@link #getIndexReader()}</li>
- * </ul>
  */
 class MultiIndex {
 
@@ -91,7 +79,12 @@
     /**
      * Names of active persistent index directories.
      */
-    private final IndexInfos indexNames = new IndexInfos();
+    private final IndexInfos indexNames = new IndexInfos("indexes");
+
+    /**
+     * Names of index directories that can be deleted.
+     */
+    private final IndexInfos deletable = new IndexInfos("deletable");
 
     /**
      * List of persistent indexes.
@@ -119,10 +112,21 @@
     private VolatileIndex volatileIndex;
 
     /**
+     * Flag indicating whether an update operation is in progress.
+     */
+    private boolean updateInProgress = false;
+
+    /**
      * If not <code>null</code> points to a valid <code>IndexReader</code> that
      * reads from all indexes, including volatile and persistent indexes.
      */
-    private IndexReader multiReader;
+    private CachingMultiReader multiReader;
+
+    /**
+     * Monitor to use to synchronize access to {@link #multiReader} and
+     * {@link #updateInProgress}.
+     */
+    private final Object updateMonitor = new Object();
 
     /**
      * <code>true</code> if the redo log contained entries on startup.
@@ -157,13 +161,17 @@
 
         this.fs = fs;
         this.handler = handler;
-        migrationCheck();
         boolean doInitialIndex = false;
-        if (fs.exists("indexes")) {
+        if (fs.exists(indexNames.getFileName())) {
             indexNames.read(fs);
         } else {
             doInitialIndex = true;
         }
+        if (fs.exists(deletable.getFileName())) {
+            deletable.read(fs);
+        }
+        // try to remove deletable files if there are any
+        attemptDelete();
 
         // read namespace mappings
         FileSystemResource mapFile = new FileSystemResource(fs, NS_MAPPING_FILE);
@@ -201,6 +209,8 @@
                             addNodePersistent(state);
                         } catch (NoSuchItemStateException e) {
                             // item does not exist anymore
+                        } catch (Exception e) {
+                            log.warn("Unable to add node to index: ", e);
                         }
                     } else {
                         deleteNodePersistent(entry.uuid);
@@ -233,50 +243,67 @@
     }
 
     /**
+     * Update the index by removing some documents and adding others.
+     *
+     * @param remove Iterator of <code>Term</code>s that identify documents to
+     *               remove
+     * @param add    Iterator of <code>Document</code>s to add. Calls to
+     *               <code>next()</code> on this iterator may return
+     *               <code>null</code>, to indicate that a node could not be
+     *               indexed successfully.
+     */
+    synchronized void update(Iterator remove, Iterator add) throws IOException {
+        synchronized (updateMonitor) {
+            updateInProgress = true;
+        }
+        boolean hasAdditions = add.hasNext();
+        try {
+            // todo block with remove & add is not atomic
+            while (remove.hasNext()) {
+                internalRemoveDocument((Term) remove.next());
+            }
+            while (add.hasNext()) {
+                Document doc = (Document) add.next();
+                if (doc != null) {
+                    internalAddDocument(doc);
+                }
+            }
+        } finally {
+            synchronized (updateMonitor) {
+                if (hasAdditions) {
+                    lastModificationTime = System.currentTimeMillis();
+                }
+                updateInProgress = false;
+                updateMonitor.notifyAll();
+                if (multiReader != null) {
+                    multiReader.close();
+                    multiReader = null;
+                }
+            }
+        }
+    }
+
+    /**
      * Adds a document to the index.
      *
      * @param doc the document to add.
      * @throws IOException if an error occurs while adding the document to the
      *                     index.
      */
-    synchronized void addDocument(Document doc) throws IOException {
-        lastModificationTime = System.currentTimeMillis();
-        multiReader = null;
-        volatileIndex.addDocument(doc);
-        if (volatileIndex.getRedoLog().getSize() >= handler.getMinMergeDocs()) {
-            log.info("Committing in-memory index");
-            commit();
-        }
+    void addDocument(Document doc) throws IOException {
+        List add = Arrays.asList(new Document[]{doc});
+        update(EmptyIterator.INSTANCE, add.iterator());
     }
 
     /**
      * Deletes the first document that matches the <code>idTerm</code>.
      *
      * @param idTerm document that match this term will be deleted.
-     * @return the number of deleted documents.
      * @throws IOException if an error occurs while deleting the document.
      */
-    synchronized int removeDocument(Term idTerm) throws IOException {
-        lastModificationTime = System.currentTimeMillis();
-        // flush multi reader if it does not have deletions yet
-        if (multiReader != null && !multiReader.hasDeletions()) {
-            multiReader = null;
-        }
-        // if the document cannot be deleted from the volatile index
-        // delete it from one of the persistent indexes.
-        int num = volatileIndex.removeDocument(idTerm);
-        if (num == 0) {
-            for (int i = indexes.size() - 1; i >= 0; i--) {
-                PersistentIndex index = (PersistentIndex) indexes.get(i);
-                num = index.removeDocument(idTerm);
-                if (num > 0) {
-                    return num;
-                }
-            }
-        } else {
-            return num;
-        }
-        return 0;
+    void removeDocument(Term idTerm) throws IOException {
+        List remove = Arrays.asList(new Term[]{idTerm});
+        update(remove.iterator(), EmptyIterator.INSTANCE);
     }
 
     /**
@@ -288,37 +315,65 @@
      * @throws IOException if an error occurs while deleting documents.
      */
     synchronized int removeAllDocuments(Term idTerm) throws IOException {
-        lastModificationTime = System.currentTimeMillis();
-        // flush multi reader if it does not have deletions yet
-        if (multiReader != null && !multiReader.hasDeletions()) {
-            multiReader = null;
+        synchronized (updateMonitor) {
+            updateInProgress = true;
         }
-        int num = volatileIndex.removeDocument(idTerm);
-        for (int i = 0; i < indexes.size(); i++) {
-            PersistentIndex index = (PersistentIndex) indexes.get(i);
-            num += index.removeDocument(idTerm);
-            index.commit();
+        int num;
+        try {
+            num = volatileIndex.removeDocument(idTerm);
+            for (int i = 0; i < indexes.size(); i++) {
+                PersistentIndex index = (PersistentIndex) indexes.get(i);
+                num += index.removeDocument(idTerm);
+                index.commit();
+            }
+        } finally {
+            synchronized (updateMonitor) {
+                updateInProgress = false;
+                updateMonitor.notifyAll();
+                if (multiReader != null) {
+                    multiReader.close();
+                    multiReader = null;
+                }
+            }
         }
         return num;
     }
 
     /**
-     * Returns an <code>IndexReader</code> that spans alls indexes of this
+     * Returns an read-only <code>IndexReader</code> that spans alls indexes of this
      * <code>MultiIndex</code>.
      *
      * @return an <code>IndexReader</code>.
      * @throws IOException if an error occurs constructing the <code>IndexReader</code>.
      */
-    synchronized IndexReader getIndexReader() throws IOException {
-        if (multiReader == null) {
-            IndexReader[] readers = new IndexReader[indexes.size() + 1];
-            for (int i = 0; i < indexes.size(); i++) {
-                readers[i] = ((PersistentIndex) indexes.get(i)).getIndexReader();
+    IndexReader getIndexReader() throws IOException {
+        synchronized (updateMonitor) {
+            if (multiReader != null) {
+                multiReader.incrementRefCount();
+                return multiReader;
+            }
+            // no reader available
+            // wait until no update is in progress
+            while (updateInProgress) {
+                try {
+                    updateMonitor.wait();
+                } catch (InterruptedException e) {
+                    throw new IOException("Interrupted while waiting to aquire reader");
+                }
+            }
+            // some other read thread might have created the reader in the
+            // meantime -> check again
+            if (multiReader == null) {
+                IndexReader[] readers = new IndexReader[indexes.size() + 1];
+                for (int i = 0; i < indexes.size(); i++) {
+                    readers[i] = ((PersistentIndex) indexes.get(i)).getReadOnlyIndexReader();
+                }
+                readers[readers.length - 1] = volatileIndex.getReadOnlyIndexReader();
+                multiReader = new CachingMultiReader(readers);
             }
-            readers[readers.length - 1] = volatileIndex.getIndexReader();
-            multiReader = new CachingMultiReader(readers);
+            multiReader.incrementRefCount();
+            return multiReader;
         }
-        return multiReader;
     }
 
     /**
@@ -329,7 +384,14 @@
         commitTimer.cancel();
 
         // commit / close indexes
-        multiReader = null;
+        if (multiReader != null) {
+            try {
+                multiReader.close();
+            } catch (IOException e) {
+                log.error("Exception while closing search index.", e);
+            }
+            multiReader = null;
+        }
         try {
             if (volatileIndex.getRedoLog().hasEntries()) {
                 commit();
@@ -374,30 +436,86 @@
     //-------------------------< internal >-------------------------------------
 
     /**
+     * Unsynchronized implementation to remove a document from the index. Note:
+     * this method will at most remove 1 (one) document from the index. This
+     * method assumes <code>idTerm</code> is unique.
+     *
+     * @param idTerm term that identifies the document to remove.
+     * @return number of documents to remove.
+     * @throws IOException if an error occurs while updating the index.
+     */
+    private int internalRemoveDocument(Term idTerm) throws IOException {
+        // if the document cannot be deleted from the volatile index
+        // delete it from one of the persistent indexes.
+        int num = volatileIndex.removeDocument(idTerm);
+        if (num == 0) {
+            for (int i = indexes.size() - 1; i >= 0; i--) {
+                PersistentIndex index = (PersistentIndex) indexes.get(i);
+                num = index.removeDocument(idTerm);
+                if (num > 0) {
+                    return num;
+                }
+            }
+        } else {
+            return num;
+        }
+        return 0;
+    }
+
+    /**
+     * Unsynchronized implementation to add a document to the index.
+     *
+     * @param doc the document to add.
+     * @throws IOException if an error occurs while adding the document to the
+     *                     index.
+     */
+    private void internalAddDocument(Document doc) throws IOException {
+        volatileIndex.addDocument(doc);
+        if (volatileIndex.getRedoLog().getSize() >= handler.getMinMergeDocs()) {
+            log.info("Committing in-memory index");
+            commit();
+        }
+    }
+
+    /**
      * Commits the volatile index to a persistent index, commits persistent
      * indexes (persist deletions) and finally merges indexes if necessary.
      *
      * @throws IOException if an error occurs.
      */
     private void commit() throws IOException {
-        // create new index folder
-        String name = indexNames.newName();
-        FileSystem sub = new BasedFileSystem(fs, name);
-        PersistentIndex index;
-        try {
-            sub.init();
-            index = new PersistentIndex(name, sub, true, handler.getAnalyzer());
-            index.setMaxMergeDocs(handler.getMaxMergeDocs());
-            index.setMergeFactor(handler.getMergeFactor());
-            index.setMinMergeDocs(handler.getMinMergeDocs());
-            index.setUseCompoundFile(handler.getUseCompoundFile());
-            indexes.add(index);
-            indexNames.addName(name);
-            indexNames.write(fs);
-        } catch (FileSystemException e) {
-            throw new IOException(e.getMessage());
+
+        // check if volatile index contains documents at all
+        if (volatileIndex.getIndexReader().numDocs() > 0) {
+            // create new index folder
+            String name = indexNames.newName();
+            FileSystem sub = new BasedFileSystem(fs, name);
+            PersistentIndex index;
+            try {
+                sub.init();
+                index = new PersistentIndex(name, sub, true, handler.getAnalyzer());
+                index.setMaxMergeDocs(handler.getMaxMergeDocs());
+                index.setMergeFactor(handler.getMergeFactor());
+                index.setMinMergeDocs(handler.getMinMergeDocs());
+                index.setUseCompoundFile(handler.getUseCompoundFile());
+            } catch (FileSystemException e) {
+                throw new IOException(e.getMessage());
+            }
+            index.mergeIndex(volatileIndex);
+
+            // if merge has been successful add index
+            try {
+                indexes.add(index);
+                indexNames.addName(name);
+                indexNames.write(fs);
+            } catch (FileSystemException e) {
+                throw new IOException(e.getMessage());
+            }
+
+            // check if obsolete indexes can be deleted
+            // todo move to other place?
+            attemptDelete();
         }
-        index.mergeIndex(volatileIndex);
 
         // commit persistent indexes
         for (int i = 0; i < indexes.size(); i++) {
@@ -521,8 +639,13 @@
                 try {
                     fs.deleteFolder(index.getName());
                 } catch (FileSystemException e) {
-                    log.warn("Unable to delete obsolete index: " + index.getName());
-                    log.error(e.toString());
+                    // try again later
+                    deletable.addName(index.getName());
+                    try {
+                        deletable.write(fs);
+                    } catch (FileSystemException e1) {
+                        throw new IOException(e.getMessage());
+                    }
                 }
             }
         }
@@ -599,8 +722,13 @@
             try {
                 fs.deleteFolder(pi.getName());
             } catch (FileSystemException e) {
-                log.warn("Unable to delete obsolete index: " + name);
-                log.error(e.toString());
+                // try again later
+                deletable.addName(pi.getName());
+                try {
+                    deletable.write(fs);
+                } catch (FileSystemException e1) {
+                    throw new IOException(e.getMessage());
+                }
             }
             indexNames.removeName(pi.getName());
             indexes.remove(i);
@@ -615,6 +743,27 @@
     }
 
     /**
+     * Attempts to delete all files recorded in {@link #deletable}.
+     */
+    private void attemptDelete() {
+        for (int i = deletable.size() - 1; i >= 0; i--) {
+            String indexName = deletable.getName(i);
+            try {
+                fs.deleteFolder(indexName);
+                deletable.removeName(i);
+            } catch (FileSystemException e) {
+                log.info("Unable to delete obsolete index: " + indexName);
+            }
+        }
+        try {
+            deletable.write(fs);
+        } catch (Exception e) {
+            // catches IOException and FileSystemException
+            log.warn("Exception while writing deletable indexes: " + e);
+        }
+    }
+
+    /**
      * Starts the commit timer that periodically checks if the volatile index
      * should be committed. The timer task will call {@link #checkCommit()}.
      */
@@ -641,40 +790,25 @@
                 if (volatileIndex.getRedoLog().hasEntries()) {
                     log.info("Committing in-memory index after being idle for " +
                             idleTime + " ms.");
-                    commit();
+                    synchronized (updateMonitor) {
+                        updateInProgress = true;
+                    }
+                    try {
+                        commit();
+                    } finally {
+                        synchronized (updateMonitor) {
+                            lastModificationTime = System.currentTimeMillis();
+                            updateMonitor.notifyAll();
+                            if (multiReader != null) {
+                                multiReader.close();
+                                multiReader = null;
+                            }
+                        }
+                    }
                 }
             } catch (IOException e) {
                 log.error("Unable to commit volatile index", e);
             }
-        }
-    }
-
-    /**
-     * <b>todo: This check will be removed when Jackrabbit 1.0 is final.</b>
-     * <p/>
-     * Checks if an old index format is present and moves it to the new
-     * subindex structure.
-     * @throws FileSystemException if an error occurs.
-     * @throws IOException if an error occurs.
-     */
-    private void migrationCheck() throws FileSystemException, IOException {
-        if (fs.exists("segments")) {
-            // move to a sub folder
-            String name = indexNames.newName();
-            fs.createFolder(name);
-            // move all files except: redo-log and ns-mappings
-            Set exclude = new HashSet();
-            exclude.add(REDO_LOG);
-            exclude.add(NS_MAPPING_FILE);
-            String[] files = fs.listFiles("/");
-            for (int i = 0; i < files.length; i++) {
-                if (exclude.contains(files[i])) {
-                    continue;
-                }
-                fs.move(files[i], name + FileSystem.SEPARATOR + files[i]);
-            }
-            indexNames.addName(name);
-            indexNames.write(fs);
         }
     }
 }

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/PersistentIndex.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/PersistentIndex.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/PersistentIndex.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/PersistentIndex.java Mon Aug 15 05:11:52 2005
@@ -18,19 +18,13 @@
 
 import org.apache.jackrabbit.core.fs.FileSystem;
 import org.apache.jackrabbit.core.fs.FileSystemException;
-import org.apache.jackrabbit.core.state.ItemStateManager;
-import org.apache.jackrabbit.core.NodeId;
 import org.apache.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.document.Document;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Iterator;
 
 /**
  * Implements a lucene index which is based on a
@@ -122,6 +116,7 @@
         getIndexWriter().addIndexes(new Directory[]{
             index.getDirectory()
         });
+        invalidateSharedReader();
     }
 
     /**
@@ -159,61 +154,5 @@
      */
     String getName() {
         return name;
-    }
-
-    /**
-     * Checks if the nodes in this index still exist in the ItemStateManager
-     * <code>mgr</code>. Nodes that do not exist in <code>mgr</code> will
-     * be deleted from the index.
-     *
-     * @param mgr the ItemStateManager.
-     */
-    void integrityCheck(ItemStateManager mgr) {
-        // List<Integer> of document numbers to delete
-        List deleted = new ArrayList();
-        IndexReader reader;
-        try {
-            reader = getIndexReader();
-            int maxDoc = reader.maxDoc();
-            for (int i = 0; i < maxDoc; i++) {
-                if (!reader.isDeleted(i)) {
-                    Document d = reader.document(i);
-                    NodeId id = new NodeId(d.get(FieldNames.UUID));
-                    if (!mgr.hasItemState(id)) {
-                        // not known to ItemStateManager
-                        deleted.add(new Integer(i));
-                        log.warn("Node " + id.getUUID() + " does not exist anymore. Will be removed from index.");
-                    }
-                }
-            }
-        } catch (IOException e) {
-            log.error("Unable to read from index: " + e);
-            return;
-        }
-
-        // now delete them
-        for (Iterator it = deleted.iterator(); it.hasNext(); ) {
-            int docNum = ((Integer) it.next()).intValue();
-            try {
-                reader.delete(docNum);
-            } catch (IOException e) {
-                log.error("Unable to delete inexistent node from index: " + e);
-            }
-        }
-
-        // commit changes on reader
-        try {
-            commit();
-        } catch (IOException e) {
-            log.error("Unable to commit index: " + e);
-        }
-    }
-
-    /**
-     * Always returns <code>true</code>.
-     * @return <code>true</code>.
-     */
-    protected boolean useCachingReader() {
-        return true;
     }
 }

Added: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java?rev=232801&view=auto
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java (added)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java Mon Aug 15 05:11:52 2005
@@ -0,0 +1,95 @@
+/*
+ * 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.query.lucene;
+
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.document.Document;
+
+import java.io.IOException;
+
+/**
+ * Wraps the lucene <code>Hits</code> object and adds a close method that allows
+ * to release resources after a query has been executed and the results have
+ * been read completely.
+ */
+class QueryHits {
+
+    /**
+     * The lucene hits we wrap.
+     */
+    private final Hits hits;
+
+    /**
+     * The IndexReader in use by the lucene hits.
+     */
+    private final IndexReader reader;
+
+    /**
+     * Number of results.
+     */
+    private final int length;
+
+    /**
+     * Creates a new <code>QueryHits</code> instance wrapping <code>hits</code>.
+     * @param hits the lucene hits.
+     * @param reader the IndexReader in use by <code>hits</code>.
+     */
+    QueryHits(Hits hits, IndexReader reader) {
+        this.hits = hits;
+        this.reader = reader;
+        this.length = hits.length();
+    }
+
+    /**
+     * Releases resources held by this hits instance.
+     *
+     * @throws IOException if an error occurs while releasing resources.
+     */
+    public final void close() throws IOException {
+        reader.close();
+    }
+
+    /**
+     * Returns the number of results.
+     * @return the number of results.
+     */
+    public final int length() {
+      return length;
+    }
+
+    /**
+     * Returns the <code>n</code><sup>th</sup> document in this QueryHits.
+     *
+     * @param n index.
+     * @return the <code>n</code><sup>th</sup> document in this QueryHits.
+     * @throws IOException if an error occurs while reading from the index.
+     */
+    public final Document doc(int n) throws IOException {
+        return hits.doc(n);
+    }
+
+    /**
+     * Returns the score for the <code>n</code><sup>th</sup> document in this
+     * QueryHits.
+     * @param n index.
+     * @return the score for the <code>n</code><sup>th</sup> document.
+     */
+    public final float score(int n) throws IOException {
+      return hits.score(n);
+    }
+}

Propchange: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryHits.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryImpl.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryImpl.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryImpl.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/QueryImpl.java Mon Aug 15 05:11:52 2005
@@ -38,7 +38,6 @@
 import org.apache.jackrabbit.core.security.AccessManager;
 import org.apache.jackrabbit.name.QName;
 import org.apache.log4j.Logger;
-import org.apache.lucene.search.Hits;
 import org.apache.lucene.search.Query;
 
 import javax.jcr.RepositoryException;
@@ -175,8 +174,9 @@
         AccessManager accessMgr = session.getAccessManager();
 
         // execute it
+        QueryHits result = null;
         try {
-            Hits result = index.executeQuery(query, orderProperties, ascSpecs);
+            result = index.executeQuery(query, orderProperties, ascSpecs);
             uuids = new ArrayList(result.length());
             scores = new ArrayList(result.length());
 
@@ -192,6 +192,14 @@
             log.error("Exception while executing query: ", e);
             uuids = Collections.EMPTY_LIST;
             scores = Collections.EMPTY_LIST;
+        } finally {
+            if (result != null) {
+                try {
+                    result.close();
+                } catch (IOException e) {
+                    log.warn("Unable to close query result: " + e);
+                }
+            }
         }
 
         // get select properties

Added: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java?rev=232801&view=auto
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java (added)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java Mon Aug 15 05:11:52 2005
@@ -0,0 +1,220 @@
+/*
+ * 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.query.lucene;
+
+import org.apache.lucene.index.FilterIndexReader;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermPositions;
+
+import java.util.BitSet;
+import java.io.IOException;
+
+/**
+ * Overwrites the methods that would modify the index and throws an
+ * {@link UnsupportedOperationException} in each of those methods. A
+ * <code>ReadOnlyIndexReader</code> will always show all documents that have
+ * not been deleted at the time when the index reader is created.
+ */
+class ReadOnlyIndexReader extends FilterIndexReader {
+
+    /**
+     * The underlying shared reader.
+     */
+    private final SharedIndexReader reader;
+
+    /**
+     * The deleted documents as initially read from the IndexReader passed
+     * in the constructor of this class.
+     */
+    private final BitSet deleted;
+
+    /**
+     * Creates a new index reader based on <code>reader</code> at
+     * <code>modificationTick</code>.
+     * @param reader the underlying <code>IndexReader</code>.
+     * @param deleted the documents that are deleted in <code>reader</code>.
+     */
+    public ReadOnlyIndexReader(SharedIndexReader reader,
+                               BitSet deleted) {
+        super(reader);
+        this.reader = reader;
+        this.deleted = deleted;
+        // register this
+        reader.addClient(this);
+    }
+
+    //---------------------< IndexReader overwrites >---------------------------
+
+    /**
+     * Returns true if document <code>n</code> has been deleted
+     * @param n the document number
+     * @return true if document <code>n</code> has been deleted
+     */
+    public boolean isDeleted(int n) {
+        return deleted.get(n);
+    }
+
+    /**
+     * @exception UnsupportedOperationException always
+     */
+    final protected void doDelete(int docNum) {
+        throw new UnsupportedOperationException("IndexReader is read-only");
+    }
+
+    /**
+     * @exception UnsupportedOperationException always
+     */
+    final protected void doUndeleteAll() {
+        throw new UnsupportedOperationException("IndexReader is read-only");
+    }
+
+    /**
+     * @exception UnsupportedOperationException always
+     */
+    final protected void doCommit() {
+        throw new UnsupportedOperationException("IndexReader is read-only");
+    }
+
+    /**
+     * Unregisters this reader from the shared index reader. Specifically, this
+     * method does <b>not</b> close the underlying index reader, because it is
+     * shared by multiple <code>ReadOnlyIndexReader</code>s.
+     * @throws IOException if an error occurs while closing the reader.
+     */
+    protected void doClose() throws IOException {
+        reader.removeClient(this);
+    }
+
+    /**
+     * Wraps the underlying <code>TermDocs</code> and filters out documents
+     * marked as deleted.<br/>
+     * If <code>term</code> is for a {@link FieldNames#UUID} field and this
+     * <code>ReadOnlyIndexReader</code> does not have such a document,
+     * {@link CachingIndexReader#EMPTY} is returned.
+     *
+     * @param term the term to enumerate the docs for.
+     * @return TermDocs for <code>term</code>.
+     * @throws IOException if an error occurs while reading from the index.
+     */
+    public TermDocs termDocs(Term term) throws IOException {
+        // do not wrap for empty TermDocs
+        TermDocs td = reader.termDocs(term);
+        if (td != CachingIndexReader.EMPTY) {
+            td = new FilteredTermDocs(td);
+        }
+        return td;
+    }
+
+    /**
+     * Wraps the underlying <code>TermDocs</code> and filters out documents
+     * marked as deleted.
+     *
+     * @return TermDocs over the whole index.
+     * @throws IOException if an error occurs while reading from the index.
+     */
+    public TermDocs termDocs() throws IOException {
+        return new FilteredTermDocs(super.termDocs());
+    }
+
+    /**
+     * Wraps the underlying <code>TermPositions</code> and filters out documents
+     * marked as deleted.
+     *
+     * @return TermPositions over the whole index.
+     * @throws IOException if an error occurs while reading from the index.
+     */
+    public TermPositions termPositions() throws IOException {
+        return new FilteredTermPositions(super.termPositions());
+    }
+
+    //----------------------< FilteredTermDocs >--------------------------------
+
+    /**
+     * Filters a wrapped TermDocs by omitting documents marked as deleted.
+     */
+    private class FilteredTermDocs extends FilterTermDocs {
+
+        /**
+         * Creates a new filtered TermDocs based on <code>in</code>.
+         *
+         * @param in the TermDocs to filter.
+         */
+        public FilteredTermDocs(TermDocs in) {
+            super(in);
+        }
+
+        /**
+         * @inheritDoc
+         */
+        public boolean next() throws IOException {
+            boolean hasNext = super.next();
+            while (hasNext && deleted.get(super.doc())) {
+                hasNext = super.next();
+            }
+            return hasNext;
+        }
+
+        /**
+         * @inheritDoc
+         */
+        public int read(int[] docs, int[] freqs) throws IOException {
+            int count;
+            for (count = 0; count < docs.length && next(); count++) {
+                docs[count] = doc();
+                freqs[count] = freq();
+            }
+            return count;
+        }
+
+        /**
+         * @inheritDoc
+         */
+        public boolean skipTo(int i) throws IOException {
+            boolean exists = super.skipTo(i);
+            while (exists && deleted.get(doc())) {
+                exists = next();
+            }
+            return exists;
+        }
+    }
+
+    //---------------------< FilteredTermPositions >----------------------------
+
+    /**
+     * Filters a wrapped TermPositions by omitting documents marked as deleted.
+     */
+    private final class FilteredTermPositions extends FilteredTermDocs
+            implements TermPositions {
+
+        /**
+         * Creates a new filtered TermPositions based on <code>in</code>.
+         *
+         * @param in the TermPositions to filter.
+         */
+        public FilteredTermPositions(TermPositions in) {
+            super(in);
+        }
+
+        /**
+         * @inheritDoc
+         */
+        public int nextPosition() throws IOException {
+            return ((TermPositions) this.in).nextPosition();
+        }
+    }
+}

Propchange: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/ReadOnlyIndexReader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/RedoLog.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/RedoLog.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/RedoLog.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/RedoLog.java Mon Aug 15 05:11:52 2005
@@ -212,7 +212,7 @@
     private void read(EntryCollector collector) throws FileSystemException {
         InputStream in = logFile.getInputStream();
         try {
-            BufferedReader reader = new BufferedReader(new InputStreamReader(logFile.getInputStream()));
+            BufferedReader reader = new BufferedReader(new InputStreamReader(in));
             String line = null;
             while ((line = reader.readLine()) != null) {
                 try {

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SearchIndex.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SearchIndex.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SearchIndex.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SearchIndex.java Mon Aug 15 05:11:52 2005
@@ -16,7 +16,6 @@
  */
 package org.apache.jackrabbit.core.query.lucene;
 
-import EDU.oswego.cs.dl.util.concurrent.FIFOReadWriteLock;
 import org.apache.jackrabbit.Constants;
 import org.apache.jackrabbit.core.ItemManager;
 import org.apache.jackrabbit.core.SessionImpl;
@@ -32,11 +31,13 @@
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Hits;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
 
 import javax.jcr.RepositoryException;
 import javax.jcr.query.InvalidQueryException;
@@ -64,11 +65,6 @@
     private final Analyzer analyzer;
 
     /**
-     * Read-write lock to synchronize access on the index.
-     */
-    private final FIFOReadWriteLock readWriteLock = new FIFOReadWriteLock();
-
-    /**
      * minMergeDocs config parameter.
      */
     private int minMergeDocs = 1000;
@@ -153,7 +149,7 @@
                             log.info(err.toString());
                         }
                     }
-                } catch (IOException e) {
+                } catch (Exception e) {
                     log.warn("Failed to run consistency check on index: " + e);
                 }
             }
@@ -169,18 +165,7 @@
      * @throws IOException if an error occurs while adding the node to the index.
      */
     public void addNode(NodeState node) throws RepositoryException, IOException {
-        Document doc = createDocument(node, getNamespaceMappings());
-        try {
-            readWriteLock.writeLock().acquire();
-        } catch (InterruptedException e) {
-            throw new RepositoryException("Failed to aquire write lock.");
-        }
-
-        try {
-            index.addDocument(doc);
-        } finally {
-            readWriteLock.writeLock().release();
-        }
+        throw new UnsupportedOperationException("addNode");
     }
 
     /**
@@ -190,19 +175,44 @@
      * the index.
      */
     public void deleteNode(String uuid) throws IOException {
-        Term idTerm = new Term(FieldNames.UUID, uuid);
-        try {
-            readWriteLock.writeLock().acquire();
-        } catch (InterruptedException e) {
-            throw new IOException("Failed to aquire write lock.");
-        }
-
-        try {
-            index.removeDocument(idTerm);
-        } finally {
-            readWriteLock.writeLock().release();
-        }
+        throw new UnsupportedOperationException("deleteNode");
+    }
 
+    /**
+     * This implementation forwards the call to
+     * {@link MultiIndex#update(java.util.Iterator, java.util.Iterator)} and
+     * transforms the two iterators to the required types.
+     *
+     * @param remove uuids of nodes to remove.
+     * @param add    NodeStates to add. Calls to <code>next()</code> on this
+     *               iterator may return <code>null</code>, to indicate that a
+     *               node could not be indexed successfully.
+     * @throws RepositoryException if an error occurs while indexing a node.
+     * @throws IOException         if an error occurs while updating the index.
+     */
+    public void updateNodes(Iterator remove, Iterator add)
+            throws RepositoryException, IOException {
+        index.update(new AbstractIteratorDecorator(remove) {
+            public Object next() {
+                String uuid = (String) super.next();
+                return new Term(FieldNames.UUID, uuid);
+            }
+        }, new AbstractIteratorDecorator(add) {
+            public Object next() {
+                NodeState state = (NodeState) super.next();
+                if (state == null) {
+                    return null;
+                }
+                Document doc = null;
+                try {
+                    doc = createDocument(state, getNamespaceMappings());
+                } catch (RepositoryException e) {
+                    log.error("Exception while creating document for node: " +
+                            state.getUUID() + ": " + e.toString());
+                }
+                return doc;
+            }
+        });
     }
 
     /**
@@ -251,46 +261,37 @@
      * @return the lucene Hits object.
      * @throws IOException if an error occurs while searching the index.
      */
-    Hits executeQuery(Query query,
+    QueryHits executeQuery(Query query,
                              QName[] orderProps,
                              boolean[] orderSpecs) throws IOException {
-        try {
-            readWriteLock.readLock().acquire();
-        } catch (InterruptedException e) {
-            throw new IOException("Unable to obtain read lock on search index.");
-        }
-
-        Hits hits = null;
-        try {
-            SortField[] sortFields = new SortField[orderProps.length];
-            for (int i = 0; i < orderProps.length; i++) {
-                String prop = null;
-                if (Constants.JCR_SCORE.equals(orderProps[i])) {
-                    // order on jcr:score does not use the natural order as
-                    // implemented in lucene. score ascending in lucene means that
-                    // higher scores are first. JCR specs that lower score values
-                    // are first.
-                    sortFields[i] = new SortField(null, SortField.SCORE, orderSpecs[i]);
-                } else {
-                    try {
-                        prop = orderProps[i].toJCRName(getNamespaceMappings());
-                    } catch (NoPrefixDeclaredException e) {
-                        // will never happen
-                    }
-                    sortFields[i] = new SortField(prop, SharedFieldSortComparator.PROPERTIES, !orderSpecs[i]);
-                }
-            }
-
-            if (sortFields.length > 0) {
-                hits = new IndexSearcher(index.getIndexReader()).search(query, new Sort(sortFields));
+        SortField[] sortFields = new SortField[orderProps.length];
+        for (int i = 0; i < orderProps.length; i++) {
+            String prop = null;
+            if (Constants.JCR_SCORE.equals(orderProps[i])) {
+                // order on jcr:score does not use the natural order as
+                // implemented in lucene. score ascending in lucene means that
+                // higher scores are first. JCR specs that lower score values
+                // are first.
+                sortFields[i] = new SortField(null, SortField.SCORE, orderSpecs[i]);
             } else {
-                hits = new IndexSearcher(index.getIndexReader()).search(query);
+                try {
+                    prop = orderProps[i].toJCRName(getNamespaceMappings());
+                } catch (NoPrefixDeclaredException e) {
+                    // will never happen
+                }
+                sortFields[i] = new SortField(prop, SharedFieldSortComparator.PROPERTIES, !orderSpecs[i]);
             }
-        } finally {
-            readWriteLock.readLock().release();
         }
 
-        return hits;
+        IndexReader reader = index.getIndexReader();
+        IndexSearcher searcher = new IndexSearcher(reader);
+        Hits hits;
+        if (sortFields.length > 0) {
+            hits = searcher.search(query, new Sort(sortFields));
+        } else {
+            hits = searcher.search(query);
+        }
+        return new QueryHits(hits, reader);
     }
 
     /**

Added: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java?rev=232801&view=auto
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java (added)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java Mon Aug 15 05:11:52 2005
@@ -0,0 +1,109 @@
+/*
+ * 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.query.lucene;
+
+import org.apache.lucene.index.FilterIndexReader;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.index.Term;
+
+import java.util.IdentityHashMap;
+import java.util.Map;
+import java.io.IOException;
+
+/**
+ * Implements an <code>IndexReader</code>, that will close when all connected
+ * clients are disconnected AND the <code>SharedIndexReader</code>s
+ * <code>close()</code> method itself has been called.
+ */
+class SharedIndexReader extends FilterIndexReader {
+
+    /**
+     * Set to <code>true</code> if this index reader should be closed, when
+     * all connected clients are disconnected.
+     */
+    private boolean closeRequested = false;
+
+    /**
+     * Map of all registered clients to this shared index reader. The Map
+     * is rather used as a Set, because each value is the same Object as its
+     * associated key.
+     */
+    private final Map clients = new IdentityHashMap();
+
+    /**
+     * Creates a new <code>SharedIndexReader</code> which is based on
+     * <code>in</code>.
+     * @param in the underlying <code>IndexReader</code>.
+     */
+    public SharedIndexReader(CachingIndexReader in) {
+        super(in);
+    }
+
+    /**
+     * Registeres <code>client</code> with this reader. As long as clients are
+     * registered, this shared reader will not release resources on {@link
+     * #close()} and will not actually close but only marks itself to close when
+     * the last client is unregistered.
+     *
+     * @param client the client to register.
+     */
+    public synchronized void addClient(Object client) {
+        clients.put(client, client);
+    }
+
+    /**
+     * Unregisters the <code>client</code> from this index reader.
+     *
+     * @param client a client of this reader.
+     * @throws IOException if an error occurs while detaching the client from
+     *                     this shared reader.
+     */
+    public synchronized void removeClient(Object client) throws IOException {
+        clients.remove(client);
+        if (clients.isEmpty() && closeRequested) {
+            super.doClose();
+        }
+    }
+
+    /**
+     * Closes this index if no client is registered, otherwise this reader is
+     * marked to close when the last client is disconnected.
+     *
+     * @throws IOException if an error occurs while closing.
+     */
+    protected synchronized void doClose() throws IOException {
+        if (clients.isEmpty()) {
+            super.doClose();
+        } else {
+            closeRequested = true;
+        }
+    }
+
+    /**
+     * Simply passes the call to the wrapped reader as is.<br/>
+     * If <code>term</code> is for a {@link FieldNames#UUID} field and this
+     * <code>SharedIndexReader</code> does not have such a document,
+     * {@link CachingIndexReader#EMPTY} is returned.
+     *
+     * @param term the term to enumerate the docs for.
+     * @return TermDocs for <code>term</code>.
+     * @throws IOException if an error occurs while reading from the index.
+     */
+    public TermDocs termDocs(Term term) throws IOException {
+        return in.termDocs(term);
+    }
+}

Propchange: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/SharedIndexReader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/VolatileIndex.java
URL: http://svn.apache.org/viewcvs/incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/VolatileIndex.java?rev=232801&r1=232800&r2=232801&view=diff
==============================================================================
--- incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/VolatileIndex.java (original)
+++ incubator/jackrabbit/trunk/core/src/java/org/apache/jackrabbit/core/query/lucene/VolatileIndex.java Mon Aug 15 05:11:52 2005
@@ -98,6 +98,7 @@
         if (pending.size() >= bufferSize) {
             commitPending();
         }
+        invalidateSharedReader();
     }
 
     /**
@@ -125,8 +126,8 @@
         } else {
             // remove document from index
             return super.getIndexReader().delete(idTerm);
+            }
         }
-    }
 
     /**
      * Overwrites the implementation in {@link AbstractIndex} to trigger