You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by eh...@apache.org on 2005/05/02 11:04:08 UTC

svn commit: r165606 - in /lucene/java/trunk/contrib/memory: ./ src/ src/java/ src/java/org/ src/java/org/apache/ src/java/org/apache/lucene/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/memory/ src/test/ src/test/org/ src/test/org/apache/ src/test/org/apache/lucene/ src/test/org/apache/lucene/index/ src/test/org/apache/lucene/index/memory/

Author: ehatcher
Date: Mon May  2 02:04:07 2005
New Revision: 165606

URL: http://svn.apache.org/viewcvs?rev=165606&view=rev
Log:
Enhancement #34585 - high-performance in-memory index contributed by Wolfgang Hoschek

Added:
    lucene/java/trunk/contrib/memory/
    lucene/java/trunk/contrib/memory/build.xml
    lucene/java/trunk/contrib/memory/src/
    lucene/java/trunk/contrib/memory/src/java/
    lucene/java/trunk/contrib/memory/src/java/org/
    lucene/java/trunk/contrib/memory/src/java/org/apache/
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/PatternAnalyzer.java
    lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/package.html
    lucene/java/trunk/contrib/memory/src/test/
    lucene/java/trunk/contrib/memory/src/test/org/
    lucene/java/trunk/contrib/memory/src/test/org/apache/
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/PatternAnalyzerTest.java
    lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/testqueries.txt

Added: lucene/java/trunk/contrib/memory/build.xml
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/build.xml?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/build.xml (added)
+++ lucene/java/trunk/contrib/memory/build.xml Mon May  2 02:04:07 2005
@@ -0,0 +1,10 @@
+<?xml version="1.0"?>
+
+<project name="memory" default="default">
+
+  <description>
+    High-performance single-document index to compare against Query
+  </description>
+
+  <import file="../contrib-build.xml"/>
+</project>

Added: lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (added)
+++ lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Mon May  2 02:04:07 2005
@@ -0,0 +1,988 @@
+package org.apache.lucene.index.memory;
+
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringReader;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.index.TermEnum;
+import org.apache.lucene.index.TermFreqVector;
+import org.apache.lucene.index.TermPositionVector;
+import org.apache.lucene.index.TermPositions;
+import org.apache.lucene.search.HitCollector;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.Similarity;
+
+/**
+ * High-performance single-document main memory Apache Lucene fulltext search index. 
+ * 
+ * <h4>Overview</h4>
+ * 
+ * This class is a replacement/substitute for a large subset of
+ * {@link org.apache.lucene.store.RAMDirectory} functionality. It is designed to
+ * enable maximum efficiency for on-the-fly matchmaking combining structured and 
+ * fuzzy full-text search in streaming applications such as Nux XQuery based XML 
+ * message queues, publish-subscribe systems for newsfeeds, data acquisition and 
+ * distribution systems, application level routers, firewalls, classifiers, etc. 
+ * For example as in <code>float score = query(String text, Query query)</code>.
+ * <p>
+ * Each instance can hold at most one Lucene "document", with a document containing
+ * zero or more "fields", each field having a name and a free text value. The
+ * free text value is tokenized (split and transformed) into zero or more index terms 
+ * (aka words) on <code>addField()</code>, according to the policy implemented by an
+ * Analyzer. For example, Lucene analyzers can split on whitespace, normalize to lower case
+ * for case insensitivity, ignore common terms with little discriminatory value such as "he", "in", "and" (stop
+ * words), reduce the terms to their natural linguistic root form such as "fishing"
+ * being reduced to "fish" (stemming), resolve synonyms/inflexions/thesauri 
+ * (upon indexing and/or querying), etc. For details, see
+ * <a target="_blank" href="http://today.java.net/pub/a/today/2003/07/30/LuceneIntro.html">Lucene Analyzer Intro</a>.
+ * <p>
+ * Arbitrary Lucene queries can be run against this class - see <a target="_blank" 
+ * href="http://lucene.apache.org/java/docs/queryparsersyntax.html">Lucene Query Syntax</a>
+ * as well as <a target="_blank" 
+ * href="http://today.java.net/pub/a/today/2003/11/07/QueryParserRules.html">Query Parser Rules</a>.
+ * Note that a Lucene query selects on the field names and associated (indexed) 
+ * tokenized terms, not on the original free text(s) - the latter are not stored 
+ * but rather thrown away immediately after tokenization.
+ * <p>
+ * For some interesting background information on search technology, see 
+ * <a target="_blank" 
+ * href="http://www.tbray.org/ongoing/When/200x/2003/07/30/OnSearchTOC">On Search, the Series</a>.
+ * 
+ * 
+ * <h4>Example Usage</h4> 
+ * 
+ * <pre>
+ * Analyzer analyzer = new PatternAnalyzer.DEFAULT_ANALYZER;
+ * //Analyzer analyzer = new SimpleAnalyzer();
+ * MemoryIndex index = new MemoryIndex();
+ * index.addField("content", "James is in the woods", analyzer);
+ * index.addField("title", "Tales of James", analyzer);
+ * float score = index.query(QueryParser.parse("woods AND title:tales", "content", analyzer));
+ * if (score &gt; 0.0f) {
+ *     System.out.println("it's a match");
+ * } else {
+ *     System.out.println("no match found");
+ * }
+ * score = index.query(QueryParser.parse("wood* AND title:tale~0.2", "content", analyzer));
+ * System.out.println("score=" + score);
+ * System.out.println("indexData=" + index.toString());
+ * </pre>
+ * 
+ * 
+ * <h4>Example XQuery Usage</h4> 
+ * 
+ * <pre>
+ * (: An XQuery that finds all books authored by James that have something to do with "fish", sorted by relevance :)
+ * declare namespace lucene = "java:nux.xom.pool.FullTextUtil";
+ * declare variable $query := "fish~"; (: any arbitrary Lucene query can go here :)
+ * 
+ * for $book in /books/book[author="James" and lucene:match(string(./abstract), $query) > 0.0]
+ * let $score := lucene:match(string($book/abstract), $query)
+ * order by $score descending
+ * return (&lt;score>{$score}</score>, $book)
+ * </pre>
+ * 
+ * 
+ * <h4>No thread safety guarantees</h4>
+ * 
+ * An instance can be queried multiple times with the same or different queries,
+ * but an instance is not thread-safe. If desired use idioms such as:
+ * <pre>
+ * MemoryIndex index = ...
+ * synchronized (index) {
+ *    // read and/or write index (i.e. add fields and/or query)
+ * } 
+ * </pre>
+ * 
+ * 
+ * <h4>Performance Notes</h4>
+ * 
+ * Internally there's a new data structure geared towards efficient indexing 
+ * and searching, plus the necessary support code to seamlessly plug into the Lucene 
+ * framework.
+ * <p>
+ * This class performs very well for very small texts (e.g. 10 chars) 
+ * as well as for large texts (e.g. 10 MB) and everything in between. 
+ * Typically, it is about 10-100 times faster than <code>RAMDirectory</code>.
+ * Note that <code>RAMDirectory</code> has particularly 
+ * large efficiency overheads for small to medium sized texts, both in time and space.
+ * Indexing a field with N tokens takes O(N) in the best case, and O(N logN) in the worst 
+ * case. Memory consumption is probably larger than for <code>RAMDirectory</code>.
+ * <p>
+ * If you're curious about
+ * the whereabouts of bottlenecks, run java 1.5 with the non-perturbing '-server
+ * -agentlib:hprof=cpu=samples,depth=10' flags, then study the trace log and
+ * correlate its hotspot trailer with its call stack headers (see <a
+ * target="_blank"
+ * href="http://java.sun.com/developer/technicalArticles/Programming/HPROF.html">
+ * hprof tracing </a>).
+ * 
+ * @author whoschek.AT.lbl.DOT.gov
+ */
+public class MemoryIndex {
+
+	/** info for each field: Map<String fieldName, Info field> */
+	private final HashMap fields = new HashMap();
+	
+	/** fields sorted ascending by fieldName; lazily computed on demand */
+	private transient Map.Entry[] sortedFields; 
+	
+	/** pos: positions[3*i], startOffset: positions[3*(i+1)], endOffset: positions[3*(i+2)] */
+	private final int stride;
+	
+	private static final long serialVersionUID = 2782195016849084649L;
+
+	private static final boolean DEBUG = false;
+	
+	/**
+	 * Sorts term entries into ascending order; also works for
+	 * Arrays.binarySearch() and Arrays.sort()
+	 */
+	private static final Comparator termComparator = new Comparator() {
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Map.Entry) { 
+				o1 = ((Map.Entry) o1).getKey();
+			}
+			if (o2 instanceof Map.Entry) { 
+				o2 = ((Map.Entry) o2).getKey();
+			}
+			return ((String) o1).compareTo((String) o2);
+		}
+	};
+
+	/**
+	 * Constructs an instance.
+	 */
+	public MemoryIndex() {
+		this(false);
+	}
+	
+	/**
+	 * Constructs an instance that can optionally store the start and end
+	 * character offset of each token term in the text. This can be useful for
+	 * highlighting of hit locations with the Lucene highlighter package.
+	 * Private until the highlighter package matures, so that this can actually
+	 * be meaningfully integrated.
+	 * 
+	 * @param storeOffsets
+	 *            whether or not to store the start and end character offset of
+	 *            each token term in the text
+	 */
+	private MemoryIndex(boolean storeOffsets) {
+		this.stride = storeOffsets ? 3 : 1;
+	}
+	
+	/**
+	 * Convenience method; Tokenizes the given field text and adds the resulting
+	 * terms to the index; Equivalent to adding a tokenized, indexed,
+	 * termVectorStored, unstored, non-keyword Lucene
+	 * {@link org.apache.lucene.document.Field}.
+	 * 
+	 * @param fieldName
+	 *            a name to be associated with the text
+	 * @param text
+	 *            the text to tokenize and index.
+	 * @param analyzer
+	 *            the analyzer to use for tokenization
+	 */
+	public void addField(String fieldName, String text, Analyzer analyzer) {
+		if (fieldName == null)
+			throw new IllegalArgumentException("fieldName must not be null");
+		if (text == null)
+			throw new IllegalArgumentException("text must not be null");
+		if (analyzer == null)
+			throw new IllegalArgumentException("analyzer must not be null");
+		
+		TokenStream stream;
+		if (analyzer instanceof PatternAnalyzer) {
+			stream = ((PatternAnalyzer) analyzer).tokenStream(fieldName, text);
+		} else {
+			stream = analyzer.tokenStream(fieldName, new StringReader(text));
+		}
+		addField(fieldName, stream);
+	}
+	
+	/**
+	 * Iterates over the given token stream and adds the resulting terms to the index;
+	 * Equivalent to adding a tokenized, indexed, termVectorStored, unstored,
+	 * Lucene {@link org.apache.lucene.document.Field}.
+	 * Finally closes the token stream. Note that untokenized keywords can be added with this method via 
+	 * the Lucene contrib <code>KeywordTokenizer</code> or similar utilities.
+	 * 
+	 * @param fieldName
+	 *            a name to be associated with the text
+	 * @param stream
+	 *            the token stream to retrieve tokens from.
+	 */
+	public void addField(String fieldName, TokenStream stream) {
+		/*
+		 * Note that this method signature avoids having a user call new
+		 * o.a.l.d.Field(...) which would be much too expensive due to the
+		 * String.intern() usage of that class.
+		 * 
+		 * More often than not, String.intern() leads to serious performance
+		 * degradations rather than improvements! If you're curious why, check
+		 * out the JDK's native code, see how it oscillates multiple times back
+		 * and forth between Java code and native code on each intern() call,
+		 * only to end up using a plain vanilla java.util.HashMap on the Java
+		 * heap for it's interned strings! String.equals() has a small cost
+		 * compared to String.intern(), trust me. Application level interning
+		 * (e.g. a HashMap per Directory/Index) typically leads to better
+		 * solutions than frequent hidden low-level calls to String.intern().
+		 * 
+		 * Perhaps with some luck, Lucene's Field.java (and Term.java) and
+		 * cousins could be fixed to not use String.intern(). Sigh :-(
+		 */
+		try {
+			if (fieldName == null)
+				throw new IllegalArgumentException("fieldName must not be null");
+			if (stream == null)
+				throw new IllegalArgumentException("token stream must not be null");
+			if (fields.get(fieldName) != null)
+				throw new IllegalArgumentException("field must not be added more than once");
+			
+			HashMap terms = new HashMap();
+			int numTokens = 0;
+			int pos = -1;
+			Token token;
+			
+			while ((token = stream.next()) != null) {
+				numTokens++;
+				pos += token.getPositionIncrement();
+				
+				String term = token.termText();
+				if (DEBUG) System.err.println("token='" + term + "'");
+				ArrayIntList positions = (ArrayIntList) terms.get(term);
+				if (positions == null) { // term not seen before
+					positions = new ArrayIntList(stride);
+					terms.put(term, positions);
+				}
+				if (stride == 1)
+					positions.add(pos);
+				else
+					positions.add(pos, token.startOffset(), token.endOffset());
+			}
+			
+			// ensure infos.numTokens > 0 invariant; needed for correct operation of terms()
+			if (numTokens > 0) {
+				fields.put(fieldName, new Info(terms, numTokens));
+				sortedFields = null; // invalidate sorted view, if any
+			}
+		} catch (IOException e) { // can never happen
+			throw new RuntimeException(e);
+		} finally {
+			try {
+				if (stream != null) stream.close();
+			} catch (IOException e2) {
+				throw new RuntimeException(e2);
+			}
+		}
+	}
+	
+	/**
+	 * Creates and returns a searcher that can be used to execute arbitrary
+	 * Lucene queries and to collect the resulting query results as hits.
+	 */
+	public IndexSearcher createSearcher() {
+		MemoryIndexReader reader = new MemoryIndexReader();
+		IndexSearcher searcher = new IndexSearcher(reader); // ensures no auto-close !!
+		reader.setSearcher(searcher); // to later get hold of searcher.getSimilarity()
+		return searcher;
+	}
+	
+	/**
+	 * Convenience method that efficiently returns the relevance score by
+	 * matching this index against the given Lucene query expression.
+	 * 
+	 * @param query
+	 *            an arbitrary Lucene query to run against this index
+	 * @return the relevance score of the matchmaking; A number in the range
+	 *         [0.0 .. 1.0], with 0.0 indicating no match. The higher the number
+	 *         the better the match.
+	 * @see org.apache.lucene.queryParser.QueryParser#parse(String, String,
+	 *      Analyzer)
+	 */
+	public float search(Query query) {
+		if (query == null) 
+			throw new IllegalArgumentException("query must not be null");
+		
+		if (fields.size() == 0) return 0.0f; // nothing to do
+		Searcher searcher = createSearcher();
+		try {
+			final float[] scores = new float[1]; // inits to 0.0f (no match)
+			searcher.search(query, new HitCollector() {
+				public void collect(int doc, float score) {
+					scores[0] = score;
+				}
+			});
+			float score = scores[0];
+			return score;
+		} catch (IOException e) { // can never happen (RAMDirectory)
+			throw new RuntimeException(e);
+		} finally {
+			// searcher.close();
+			/*
+			 * Note that it is harmless and important for good performance to
+			 * NOT close the index reader!!! This avoids all sorts of
+			 * unnecessary baggage and locking in the Lucene IndexReader
+			 * superclass, all of which is completely unnecessary for this main
+			 * memory index data structure without thread-safety claims.
+			 * 
+			 * Wishing IndexReader would be an interface...
+			 * 
+			 * Actually with the new tight createSearcher() API auto-closing is now
+			 * made impossible, hence searcher.close() would be harmless...
+			 */
+		}		
+	}
+	
+	/**
+	 * Returns a reasonable approximation of the main memory [bytes] consumed by
+	 * this instance. Useful for smart memory sensititve caches/pools. Assumes
+	 * fieldNames are interned, whereas tokenized terms are memory-overlaid. For
+	 * simplicity, assumes no VM word boundary alignment of instance vars.
+	 */
+	public int getMemorySize() {
+		// for example usage in a smart cache see nux.xom.pool.Pool
+		int HEADER = 12; // object header of any java object
+		int PTR = 4; // pointer on 32 bit VMs
+		int ARR = HEADER + 4;
+		int STR = HEADER + 3*4 + PTR + ARR; // string
+		int INTARRLIST = HEADER + 4 + PTR + ARR;
+		int HASHMAP = HEADER + 4*PTR + 4*4 + ARR;
+		
+		int size = 0;
+		size += HEADER + 3*PTR; // memory index
+		if (sortedFields != null) size += ARR + PTR * sortedFields.length;
+		
+		size += HASHMAP + fields.size() * (PTR + HEADER + 3*PTR + 4); // Map.entries
+		Iterator iter = fields.entrySet().iterator();
+		while (iter.hasNext()) { // for each Field Info
+			Map.Entry entry = (Map.Entry) iter.next();			
+			Info info = (Info) entry.getValue();
+			size += HEADER + 4 + PTR + PTR; // Info instance vars
+			if (info.sortedTerms != null) size += ARR + PTR * info.sortedTerms.length;
+			
+			int len = info.terms.size();
+			size += HASHMAP + len * (PTR + HEADER + 3*PTR + 4); // Map.entries
+			Iterator iter2 = info.terms.entrySet().iterator();
+			while (--len >= 0) { // for each term
+				Map.Entry e = (Map.Entry) iter2.next();
+				size += STR - ARR; // assumes substring() memory overlay
+//				size += STR + 2 * ((String) e.getKey()).length();
+				ArrayIntList positions = (ArrayIntList) e.getValue();
+				size += INTARRLIST + 4*positions.size();
+			}
+		}
+		return size;
+	}	
+
+	private int numPositions(ArrayIntList positions) {
+		return positions.size() / stride;
+	}
+	
+	/** sorts into ascending order (on demand), reusing memory along the way */
+	private void sortFields() {
+		if (sortedFields == null) sortedFields = sort(fields);
+	}
+	
+	/** returns a view of the given map's entries, sorted ascending by key */
+	private static Map.Entry[] sort(HashMap map) {
+		int size = map.size();
+		Map.Entry[] entries = new Map.Entry[size];
+		
+		Iterator iter = map.entrySet().iterator();
+		for (int i=0; i < size; i++) {
+			entries[i] = (Map.Entry) iter.next();
+		}
+		
+		if (size > 1) Arrays.sort(entries, termComparator);
+		return entries;
+	}
+	
+	/** Returns a String representation of the index data for debugging purposes. */
+	public String toString() {
+		StringBuffer result = new StringBuffer(256);		
+		sortFields();		
+		int sumChars = 0;
+		int sumPositions = 0;
+		int sumTerms = 0;
+		
+		for (int i=0; i < sortedFields.length; i++) {
+			Map.Entry entry = sortedFields[i];
+			String fieldName = (String) entry.getKey();
+			Info info = (Info) entry.getValue();
+			info.sortTerms();
+			result.append(fieldName + ":\n");
+			
+			int numChars = 0;
+			int numPositions = 0;
+			for (int j=0; j < info.sortedTerms.length; j++) {
+				Map.Entry e = info.sortedTerms[j];
+				String term = (String) e.getKey();
+				ArrayIntList positions = (ArrayIntList) e.getValue();
+				result.append("\t'" + term + "':" + numPositions(positions) + ":");
+				result.append(positions.toString(stride)); // ignore offsets
+				result.append("\n");
+				numPositions += numPositions(positions);
+				numChars += term.length();
+			}
+			
+			result.append("\tterms=" + info.sortedTerms.length);
+			result.append(", positions=" + numPositions);
+			result.append(", Kchars=" + (numChars/1000.0f));
+			result.append("\n");
+			sumPositions += numPositions;
+			sumChars += numChars;
+			sumTerms += info.sortedTerms.length;
+		}
+		
+		result.append("\nfields=" + sortedFields.length);
+		result.append(", terms=" + sumTerms);
+		result.append(", positions=" + sumPositions);
+		result.append(", Kchars=" + (sumChars/1000.0f));
+		return result.toString();
+	}
+	
+	
+	///////////////////////////////////////////////////////////////////////////////
+	// Nested classes:
+	///////////////////////////////////////////////////////////////////////////////
+	/**
+	 * Index data structure for a field; Contains the tokenized term texts and
+	 * their positions.
+	 */
+	private static final class Info implements Serializable {
+		
+		/**
+		 * Term strings and their positions for this field: Map <String
+		 * termText, ArrayIntList positions>
+		 */
+		private final HashMap terms; 
+		
+		/** Terms sorted ascending by term text; computed on demand */
+		private transient Map.Entry[] sortedTerms;
+		
+		/** Number of added tokens for this field */
+		private final int numTokens;
+
+		private static final long serialVersionUID = 2882195016849084649L;	
+
+		public Info(HashMap terms, int numTokens) {
+			this.terms = terms;
+			this.numTokens = numTokens;
+		}
+		
+		/**
+		 * Sorts hashed terms into ascending order, reusing memory along the
+		 * way. Note that sorting is lazily delayed until required (often it's
+		 * not required at all). If a sorted view is required then hashing +
+		 * sort + binary search is still faster and smaller than TreeMap usage
+		 * (which would be an alternative and somewhat more elegant approach,
+		 * apart from more sophisticated Tries / prefix trees).
+		 */
+		public void sortTerms() {
+			if (sortedTerms == null) sortedTerms = sort(terms);
+		}
+				
+		/** note that the frequency can be calculated as numPosition(getPositions(x)) */
+		public ArrayIntList getPositions(String term) {
+			return (ArrayIntList) terms.get(term);
+		}
+
+		/** note that the frequency can be calculated as numPosition(getPositions(x)) */
+		public ArrayIntList getPositions(int pos) {
+			return (ArrayIntList) sortedTerms[pos].getValue();
+		}
+		
+	}
+	
+	
+	///////////////////////////////////////////////////////////////////////////////
+	// Nested classes:
+	///////////////////////////////////////////////////////////////////////////////
+	/**
+	 * Efficient resizable auto-expanding list holding <code>int</code> elements;
+	 * implemented with arrays.
+	 */
+	private static final class ArrayIntList implements Serializable {
+
+		private int[] elements;
+		private int size = 0;
+		
+		private static final long serialVersionUID = 2282195016849084649L;	
+			
+		public ArrayIntList() {
+			this(10);
+		}
+
+		public ArrayIntList(int initialCapacity) {
+			elements = new int[initialCapacity];
+		}
+
+		public void add(int elem) {
+			if (size == elements.length) ensureCapacity(size + 1);
+			elements[size++] = elem;
+		}
+
+		public void add(int pos, int start, int end) {
+			if (size + 3 > elements.length) ensureCapacity(size + 3);
+			elements[size] = pos;
+			elements[size+1] = start;
+			elements[size+2] = end;
+			size += 3;
+		}
+
+		public int get(int index) {
+			if (index >= size) throwIndex(index);
+			return elements[index];
+		}
+		
+		public int size() {
+			return size;
+		}
+		
+		public int[] toArray(int stride) {
+			int[] arr = new int[size() / stride];
+			if (stride == 1)
+				System.arraycopy(elements, 0, arr, 0, size); // fast path
+			else 
+				for (int i=0, j=0; j < size; i++, j += stride) arr[i] = elements[j];
+			return arr;
+		}
+		
+		private void ensureCapacity(int minCapacity) {
+			int newCapacity = Math.max(minCapacity, (elements.length * 3) / 2 + 1);
+			int[] newElements = new int[newCapacity];
+			System.arraycopy(elements, 0, newElements, 0, size);
+			elements = newElements;
+		}
+
+		private void throwIndex(int index) {
+			throw new IndexOutOfBoundsException("index: " + index
+						+ ", size: " + size);
+		}
+		
+		/** returns the first few positions (without offsets); debug only */
+		public String toString(int stride) {
+			int s = size() / stride;
+			int len = Math.min(10, s); // avoid printing huge lists
+			StringBuffer buf = new StringBuffer(4*len);
+			buf.append("[");
+			for (int i = 0; i < len; i++) {
+				buf.append(get(i*stride));
+				if (i < len-1) buf.append(", ");
+			}
+			if (len != s) buf.append(", ..."); // and some more...
+			buf.append("]");
+			return buf.toString();
+		}		
+	}
+	
+	
+	///////////////////////////////////////////////////////////////////////////////
+	// Nested classes:
+	///////////////////////////////////////////////////////////////////////////////
+	private static final Term MATCH_ALL_TERM = new Term("", "");
+		
+	/**
+	 * Search support for Lucene framework integration; implements all methods
+	 * required by the Lucene IndexReader contracts.
+	 */
+	private final class MemoryIndexReader extends IndexReader {
+		
+		private Searcher searcher; // needed to find searcher.getSimilarity() 
+		
+		private MemoryIndexReader() {
+			super(null); // avoid as much superclass baggage as possible
+		}
+		
+		// lucene >= 1.9 or lucene-1.4.3 with patch removing "final" in superclass
+		protected void finalize() {}
+		
+		private Info getInfo(String fieldName) {
+			return (Info) fields.get(fieldName);
+		}
+		
+		private Info getInfo(int pos) {
+			return (Info) sortedFields[pos].getValue();
+		}
+		
+		public int docFreq(Term term) {
+			Info info = getInfo(term.field());
+			int freq = 0;
+			if (info != null) freq = info.getPositions(term.text()) != null ? 1 : 0;
+			if (DEBUG) System.err.println("MemoryIndexReader.docFreq: " + term + ", freq:" + freq);
+			return freq;
+		}
+	
+		public TermEnum terms() {
+			if (DEBUG) System.err.println("MemoryIndexReader.terms()");
+			return terms(MATCH_ALL_TERM);
+		}
+		
+		public TermEnum terms(Term term) {
+			if (DEBUG) System.err.println("MemoryIndexReader.terms: " + term);
+	
+			int i; // index into info.sortedTerms
+			int j; // index into sortedFields
+			
+			sortFields();
+			j = Arrays.binarySearch(sortedFields, term.field(), termComparator);
+			if (j < 0) { // not found; choose successor
+				j = -j -1; 
+				i = 0;
+				if (j < sortedFields.length) getInfo(j).sortTerms();
+			}
+			else { // found
+				Info info = getInfo(j);
+				info.sortTerms();
+				i = Arrays.binarySearch(info.sortedTerms, term.text(), termComparator);
+				if (i < 0) { // not found; choose successor
+					i = -i -1;
+					if (i >= info.sortedTerms.length) { // move to next successor
+						j++;
+						i = 0;
+						if (j < sortedFields.length) getInfo(j).sortTerms();
+					}
+				}
+			}
+			final int ix = i;
+			final int jx = j;
+	
+			return new TermEnum() {
+	
+				private int i = ix; // index into info.sortedTerms
+				private int j = jx; // index into sortedFields
+	
+				public boolean next() {
+					if (DEBUG) System.err.println("TermEnum.next");
+					if (j >= sortedFields.length) return false;
+					Info info = getInfo(j);
+					if (++i < info.sortedTerms.length) return true;
+	
+					// move to successor
+					j++;
+					i = 0;
+					if (j >= sortedFields.length) return false;
+					info.sortTerms();
+					return true;
+				}
+	
+				public Term term() {
+					if (DEBUG) System.err.println("TermEnum.term: " + i);
+					if (j >= sortedFields.length) return null;
+					Info info = getInfo(j);
+					if (i >= info.sortedTerms.length) return null;
+					String fieldName = (String) sortedFields[j].getKey();
+					return new Term(fieldName, (String) info.sortedTerms[i].getKey());
+				}
+	
+				public int docFreq() {
+					if (DEBUG) System.err.println("TermEnum.docFreq");
+					if (j >= sortedFields.length) return 0;
+					Info info = getInfo(j);
+					if (i >= info.sortedTerms.length) return 0;
+					return numPositions(info.getPositions(i));
+				}
+	
+				public void close() {
+					if (DEBUG) System.err.println("TermEnum.close");
+				}
+			};
+		}
+	
+		public TermPositions termPositions() {
+			if (DEBUG) System.err.println("MemoryIndexReader.termPositions");
+			
+			return new TermPositions() {
+	
+				private boolean hasNext;
+				private int cursor = 0;
+				private ArrayIntList current;
+				
+				public void seek(Term term) {
+					if (DEBUG) System.err.println(".seek: " + term);
+					Info info = getInfo(term.field());
+					current = info == null ? null : info.getPositions(term.text());
+					hasNext = (current != null);
+					cursor = 0;
+				}
+	
+				public void seek(TermEnum termEnum) {
+					seek(termEnum.term());
+				}
+	
+				public int doc() {
+					if (DEBUG) System.err.println(".doc");
+					return 0;
+				}
+	
+				public int freq() {
+					int freq = current != null ? numPositions(current) : 0;
+					if (DEBUG) System.err.println(".freq: " + freq);
+					return freq;
+				}
+	
+				public boolean next() {
+					if (DEBUG) System.err.println(".next: " + current + ", oldHasNext=" + hasNext);
+					boolean next = hasNext;
+					hasNext = false;
+					return next;
+				}
+	
+				public int read(int[] docs, int[] freqs) {
+					if (DEBUG) System.err.println(".read: " + docs.length);
+					if (!hasNext) return 0;
+					hasNext = false;
+					docs[0] = 0;
+					freqs[0] = freq();
+					return 1;
+				}
+	
+				public boolean skipTo(int target) {
+					if (DEBUG) System.err.println(".skipTo");
+					return false;
+				}
+	
+				public void close() {
+					if (DEBUG) System.err.println(".close");
+				}
+				
+				public int nextPosition() { // implements TermPositions
+					int pos = current.get(cursor);
+					cursor += stride;
+					if (DEBUG) System.err.println(".nextPosition: " + pos);
+					return pos;
+				}
+			};
+		}
+	
+		public TermDocs termDocs() {
+			if (DEBUG) System.err.println("MemoryIndexReader.termDocs");
+			return termPositions();
+		}
+	
+		public TermFreqVector[] getTermFreqVectors(int docNumber) {
+			if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVectors");
+			TermFreqVector[] vectors = new TermFreqVector[fields.size()];
+			Iterator iter = fields.keySet().iterator();
+			for (int i=0; i < vectors.length; i++) {
+				String fieldName = (String) iter.next();
+				vectors[i] = getTermFreqVector(docNumber, fieldName);
+			}
+			return vectors;
+		}
+		
+		public TermFreqVector getTermFreqVector(int docNumber, final String fieldName) {
+			if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVector");
+			final Info info = getInfo(fieldName);
+			if (info == null) return null; // TODO: or return empty vector impl???
+			info.sortTerms();
+			
+			return new TermPositionVector() { 
+	
+				final Map.Entry[] sortedTerms = info.sortedTerms;
+				
+				public String getField() {
+					return fieldName;
+				}
+	
+				public int size() {
+					return sortedTerms.length;
+				}
+	
+				public String[] getTerms() {
+					String[] terms = new String[sortedTerms.length];
+					for (int i=0; i < sortedTerms.length; i++) {
+						terms[i] = (String) sortedTerms[i].getKey();
+					}
+					return terms;
+				}
+	
+				public int[] getTermFrequencies() {
+					int[] freqs = new int[sortedTerms.length];
+					for (int i=0; i < sortedTerms.length; i++) {
+						freqs[i] = numPositions((ArrayIntList) sortedTerms[i].getValue());
+					}
+					return freqs;
+				}
+	
+				public int indexOf(String term) {
+					int i = Arrays.binarySearch(sortedTerms, term, termComparator);
+					return i >= 0 ? i : -1;
+				}
+	
+				public int[] indexesOf(String[] terms, int start, int len) {
+					int[] indexes = new int[len];
+					for (int i=0; i < len; i++) {
+						indexes[i] = indexOf(terms[start++]);
+					}
+					return indexes;
+				}
+				
+				// lucene >= 1.4.3
+				public int[] getTermPositions(int index) {
+					return ((ArrayIntList) sortedTerms[index].getValue()).toArray(stride);
+				} 
+				
+				// lucene >= 1.9 (remove this method for lucene-1.4.3)
+				public org.apache.lucene.index.TermVectorOffsetInfo[] getOffsets(int index) {
+					if (stride == 1) return null; // no offsets stored
+					
+					ArrayIntList positions = (ArrayIntList) sortedTerms[index].getValue();
+					int size = positions.size();
+					org.apache.lucene.index.TermVectorOffsetInfo[] offsets = 
+						new org.apache.lucene.index.TermVectorOffsetInfo[size / stride];
+					
+					for (int i=0, j=1; j < size; i++, j += stride) {
+						int start = positions.get(j);
+						int end = positions.get(j+1);
+						offsets[i] = new org.apache.lucene.index.TermVectorOffsetInfo(start, end);
+					}
+					return offsets;
+				}
+
+			};
+		}
+
+		private Similarity getSimilarity() {
+			return searcher.getSimilarity();
+		}
+		
+		private void setSearcher(Searcher searcher) {
+			this.searcher = searcher;
+		}
+		
+		public byte[] norms(String fieldName) {
+			if (DEBUG) System.err.println("MemoryIndexReader.norms: " + fieldName);
+			Info info = getInfo(fieldName);
+			int numTokens = info != null ? info.numTokens : 0;
+			byte norm = Similarity.encodeNorm(getSimilarity().lengthNorm(fieldName, numTokens));
+			return new byte[] {norm};
+		}
+	
+		public void norms(String fieldName, byte[] bytes, int offset) {
+			if (DEBUG) System.err.println("MemoryIndexReader.norms: " + fieldName + "*");
+			byte[] norms = norms(fieldName);
+			System.arraycopy(norms, 0, bytes, offset, norms.length);
+		}
+	
+		protected void doSetNorm(int doc, String fieldName, byte value) {
+			throw new UnsupportedOperationException();
+		}
+	
+		public int numDocs() {
+			if (DEBUG) System.err.println("MemoryIndexReader.numDocs");
+			return fields.size() > 0 ? 1 : 0;
+		}
+	
+		public int maxDoc() {
+			if (DEBUG) System.err.println("MemoryIndexReader.maxDoc");
+			return 1;
+		}
+	
+		public Document document(int n) {
+			if (DEBUG) System.err.println("MemoryIndexReader.document");
+			return new Document(); // there are no stored fields
+		}
+	
+		public boolean isDeleted(int n) {
+			if (DEBUG) System.err.println("MemoryIndexReader.isDeleted");
+			return false;
+		}
+	
+		public boolean hasDeletions() {
+			if (DEBUG) System.err.println("MemoryIndexReader.hasDeletions");
+			return false;
+		}
+	
+		protected void doDelete(int docNum) {
+			throw new UnsupportedOperationException();
+		}
+	
+		protected void doUndeleteAll() {
+			throw new UnsupportedOperationException();
+		}
+	
+		protected void doCommit() {
+			if (DEBUG) System.err.println("MemoryIndexReader.doCommit");
+		}
+	
+		protected void doClose() {
+			if (DEBUG) System.err.println("MemoryIndexReader.doClose");
+		}
+	
+		// lucene <= 1.4.3
+		public Collection getFieldNames() {
+			if (DEBUG) System.err.println("MemoryIndexReader.getFieldNames");
+			return getFieldNames(true);
+		}
+	
+		// lucene <= 1.4.3
+		public Collection getFieldNames(boolean indexed) {
+			if (DEBUG) System.err.println("MemoryIndexReader.getFieldNames " + indexed);
+			return indexed ? Collections.unmodifiableSet(fields.keySet()) : Collections.EMPTY_SET;
+		}
+	
+		// lucene <= 1.4.3
+		public Collection getIndexedFieldNames(boolean storedTermVector) {
+			if (DEBUG) System.err.println("MemoryIndexReader.getIndexedFieldNames " + storedTermVector);
+			return getFieldNames(storedTermVector);
+		}
+	
+		// lucene >= 1.9 (deprecated) (remove this method for lucene-1.4.3)
+		public Collection getIndexedFieldNames(org.apache.lucene.document.Field.TermVector tvSpec) {
+			throw new UnsupportedOperationException(
+				"Deprecated; replaced by getFieldNames(IndexReader.FieldOption)");
+		}
+
+		// lucene >= 1.9 (remove this method for lucene-1.4.3)
+		public Collection getFieldNames(FieldOption fieldOption) {
+			if (DEBUG) System.err.println("MemoryIndexReader.getFieldNamesOption");
+			if (fieldOption == FieldOption.UNINDEXED) 
+				return Collections.EMPTY_SET;
+			if (fieldOption == FieldOption.INDEXED_NO_TERMVECTOR) 
+				return Collections.EMPTY_SET;
+			if (fieldOption == FieldOption.TERMVECTOR_WITH_OFFSET && stride == 1) 
+				return Collections.EMPTY_SET;
+			if (fieldOption == FieldOption.TERMVECTOR_WITH_POSITION_OFFSET && stride == 1) 
+				return Collections.EMPTY_SET;
+			
+			return Collections.unmodifiableSet(fields.keySet());
+		}
+	}
+
+}

Added: lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/PatternAnalyzer.java
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/PatternAnalyzer.java?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/PatternAnalyzer.java (added)
+++ lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/PatternAnalyzer.java Mon May  2 02:04:07 2005
@@ -0,0 +1,343 @@
+package org.apache.lucene.index.memory;
+
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * 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.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+import java.util.Locale;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.StopAnalyzer;
+import org.apache.lucene.analysis.StopFilter;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+
+/**
+ * Efficient Lucene analyzer/tokenizer that preferably operates on a String rather than a
+ * {@link java.io.Reader}, that can flexibly separate on a regular expression {@link Pattern}
+ * (with behaviour identical to {@link String#split(String)}),
+ * and that combines the functionality of
+ * {@link org.apache.lucene.analysis.LetterTokenizer},
+ * {@link org.apache.lucene.analysis.LowerCaseTokenizer},
+ * {@link org.apache.lucene.analysis.WhitespaceTokenizer},
+ * {@link org.apache.lucene.analysis.StopFilter} into a single efficient
+ * multi-purpose class.
+ * <p>
+ * If you are unsure how exactly a regular expression should look like, consider 
+ * prototyping by simply trying various expressions on some test texts via
+ * {@link String#split(String)}. Once you are satisfied, give that regex to 
+ * PatternAnalyzer. Also see <a target="_blank" 
+ * href="http://java.sun.com/docs/books/tutorial/extra/regex/">Java Regular Expression Tutorial</a>.
+ * <p>
+ * This class can be considerably faster than the "normal" Lucene tokenizers. 
+ * It can also serve as a building block in a compound Lucene
+ * {@link org.apache.lucene.analysis.TokenFilter} chain. For example as in this 
+ * stemming example:
+ * <pre>
+ * PatternAnalyzer pat = ...
+ * TokenStream tokenStream = new SnowballFilter(
+ *     pat.tokenStream("content", "James is running round in the woods"), 
+ *     "English"));
+ * </pre>
+ * 
+ * @author whoschek.AT.lbl.DOT.gov
+ * @author $Author: hoschek3 $
+ * @version $Revision: 1.10 $, $Date: 2005/04/29 08:51:02 $
+ */
+public class PatternAnalyzer extends Analyzer {
+	
+	/** <code>"\\W+"</code>; Divides text at non-letters (Character.isLetter(c)) */
+	public static final Pattern NON_WORD_PATTERN = Pattern.compile("\\W+");
+	
+	/** <code>"\\s+"</code>; Divides text at whitespaces (Character.isWhitespace(c) */
+	public static final Pattern WHITESPACE_PATTERN = Pattern.compile("\\s+");
+	
+	/**
+	 * A lower-casing word analyzer with English stop words (can be shared
+	 * freely across threads without harm); global per class loader.
+	 */
+	public static final PatternAnalyzer DEFAULT_ANALYZER = new PatternAnalyzer(
+		NON_WORD_PATTERN, true, StopFilter.makeStopSet(StopAnalyzer.ENGLISH_STOP_WORDS));
+		
+	private final Pattern pattern;
+	private final boolean toLowerCase;
+	private final Set stopWords;
+	
+	/**
+	 * Constructs a new instance with the given parameters.
+	 * 
+	 * @param pattern
+	 *            a regular expression delimiting tokens
+	 * @param toLowerCase
+	 *            if <code>true</code> returns tokens after applying
+	 *            String.toLowerCase()
+	 * @param stopWords
+	 *            if non-null, ignores all tokens that are contained in the
+	 *            given stop set (after previously having applied toLowerCase()
+	 *            if applicable). For example, created via
+	 *            {@link StopFilter#makeStopSet(String[])}and/or
+	 *            {@link org.apache.lucene.analysis.WordlistLoader}.
+	 */
+	public PatternAnalyzer(Pattern pattern, boolean toLowerCase, Set stopWords) {
+		if (pattern == null) 
+			throw new IllegalArgumentException("pattern must not be null");
+		
+		if (eqPattern(NON_WORD_PATTERN, pattern)) pattern = NON_WORD_PATTERN;
+		else if (eqPattern(WHITESPACE_PATTERN, pattern)) pattern = WHITESPACE_PATTERN;
+		
+		this.pattern = pattern;
+		this.toLowerCase = toLowerCase;
+		this.stopWords = stopWords;
+	}
+	
+	/**
+	 * Creates a token stream that tokenizes the given string into token terms
+	 * (aka words).
+	 * 
+	 * @param fieldName
+	 *            the name of the field to tokenize (currently ignored).
+	 * @param text
+	 *            the string to tokenize
+	 */
+	public TokenStream tokenStream(String fieldName, String text) {
+		// Ideally the Analyzer superclass should have a method with the same signature, 
+		// with a default impl that simply delegates to the StringReader flavour. 
+		if (text == null) 
+			throw new IllegalArgumentException("text must not be null");
+		
+		TokenStream stream;
+		if (pattern == NON_WORD_PATTERN) { // fast path
+			stream = new FastStringTokenizer(text, true, toLowerCase, stopWords);
+		}
+		else if (pattern == WHITESPACE_PATTERN) { // fast path
+			stream = new FastStringTokenizer(text, false, toLowerCase, stopWords);
+		}
+		else {
+			stream = new PatternTokenizer(text, pattern, toLowerCase);
+			if (stopWords != null) stream = new StopFilter(stream, stopWords);
+		}
+		
+		return stream;
+	}
+	
+	/**
+	 * Creates a token stream that tokenizes all the text in the given Reader;
+	 * This implementation forwards to <code>tokenStream(String, String)</code> and is
+	 * less efficient than <code>tokenStream(String, String)</code>.
+	 */
+	public TokenStream tokenStream(String fieldName, Reader reader) {
+		try {
+			String text = toString(reader);
+			return tokenStream(fieldName, text);
+		} catch (IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+	
+	/**  Indicates whether some other object is "equal to" this one. */
+	public boolean equals(Object other) {
+		if (this == other) return true;
+		if (other instanceof PatternAnalyzer) {
+			PatternAnalyzer p2 = (PatternAnalyzer) other;
+			return 
+				toLowerCase == p2.toLowerCase &&
+				eqPattern(pattern, p2.pattern) &&
+				eq(stopWords, p2.stopWords);
+		}
+		return false;
+	}
+	
+	/** Returns a hash code value for the object. */
+	public int hashCode() {
+		if (this == DEFAULT_ANALYZER) return -1218418418; // fast path
+		int h = 1;
+		h = 31*h + pattern.pattern().hashCode();
+		h = 31*h + pattern.flags();
+		h = 31*h + (toLowerCase ? 1231 : 1237);
+		h = 31*h + (stopWords != null ? stopWords.hashCode() : 0);
+		return h;
+	}
+	
+	/** equality where o1 and/or o2 can be null */
+	private static boolean eq(Object o1, Object o2) {
+		return (o1 == o2) || (o1 != null ? o1.equals(o2) : false);
+	}
+	
+	/** assumes p1 and p2 are not null */
+	private static boolean eqPattern(Pattern p1, Pattern p2) {
+		return p1.flags() == p2.flags() && p1.pattern().equals(p2.pattern());
+	}
+		
+	/**
+	 * Reads until end-of-stream and returns all read chars, finally closes the stream.
+	 * 
+	 * @param input the input stream
+	 * @throws IOException if an I/O error occurs while reading the stream
+	 */
+	private static String toString(Reader input) throws IOException {
+		try {
+			int len = 256;
+			char[] buffer = new char[len];
+			char[] output = new char[len];
+			
+			len = 0;
+			int n;
+			while ((n = input.read(buffer)) >= 0) {
+				if (len + n > output.length) { // grow capacity
+					char[] tmp = new char[Math.max(output.length << 1, len + n)];
+					System.arraycopy(output, 0, tmp, 0, len);
+					System.arraycopy(buffer, 0, tmp, len, n);
+					buffer = output; // use larger buffer for future larger bulk reads
+					output = tmp;
+				} else {
+					System.arraycopy(buffer, 0, output, len, n);
+				}
+				len += n;
+			}
+
+			return new String(output, 0, output.length);
+		} finally {
+			if (input != null) input.close();
+		}
+	}
+		
+	
+	///////////////////////////////////////////////////////////////////////////////
+	// Nested classes:
+	///////////////////////////////////////////////////////////////////////////////
+	/**
+	 * The work horse; performance isn't fantastic, but it's not nearly as bad
+	 * as one might think - kudos to the Sun regex developers.
+	 */
+	private static final class PatternTokenizer extends TokenStream {
+		
+		private final String str;
+		private final boolean toLowerCase;
+		private Matcher matcher;
+		private int pos = 0;
+		private static final Locale locale = Locale.getDefault();
+		
+		public PatternTokenizer(String str, Pattern pattern, boolean toLowerCase) {
+			this.str = str;
+			this.matcher = pattern.matcher(str);
+			this.toLowerCase = toLowerCase;
+		}
+
+		public Token next() {
+			if (matcher == null) return null;
+			
+			while (true) { // loop takes care of leading and trailing boundary cases
+				int start = pos;
+				int end;
+				boolean isMatch = matcher.find();
+				if (isMatch) {
+					end = matcher.start();
+					pos = matcher.end();
+				} else { 
+					end = str.length();
+					matcher = null; // we're finished
+				}
+				
+				if (start != end) { // non-empty match (header/trailer)
+					String text = str.substring(start, end);
+					if (toLowerCase) text = text.toLowerCase(locale);
+					return new Token(text, start, end);
+				}
+				if (!isMatch) return null;
+			}
+		}
+		
+	}	
+	
+	
+	///////////////////////////////////////////////////////////////////////////////
+	// Nested classes:
+	///////////////////////////////////////////////////////////////////////////////
+	/**
+	 * Special-case class for best perfomance in common cases; this class is
+	 * otherwise unnecessary.
+	 */
+	private static final class FastStringTokenizer extends TokenStream {
+		
+		private final String str;
+		private int pos;
+		private final boolean isLetter;
+		private final boolean toLowerCase;
+		private final Set stopWords;
+		private static final Locale locale = Locale.getDefault();
+		
+		public FastStringTokenizer(String str, boolean isLetter, boolean toLowerCase, Set stopWords) {
+			this.str = str;
+			this.isLetter = isLetter;
+			this.toLowerCase = toLowerCase;
+			this.stopWords = stopWords;
+		}
+
+		public Token next() {
+			// cache loop instance vars (performance)
+			String s = str;
+			int len = s.length();
+			int i = pos;
+			boolean letter = isLetter;
+			
+			int start = 0;
+			String text;
+			do {
+				// find beginning of token
+				text = null;
+				while (i < len && !isTokenChar(s.charAt(i), letter)) {
+					i++;
+				}
+				
+				if (i < len) { // found beginning; now find end of token
+					start = i;
+					while (i < len && isTokenChar(s.charAt(i), letter)) {
+						i++;
+					}
+					
+					text = s.substring(start, i);
+					if (toLowerCase) text = text.toLowerCase(locale);
+//					if (toLowerCase) {						
+////						use next line once JDK 1.5 String.toLowerCase() performance regression is fixed
+//						text = s.substring(start, i).toLowerCase(); 
+////						char[] chars = new char[i-start];
+////						for (int j=start; j < i; j++) chars[j-start] = Character.toLowerCase(s.charAt(j));
+////						text = new String(chars);
+//					} else {
+//						text = s.substring(start, i);
+//					}
+				}
+			} while (text != null && isStopWord(text));
+			
+			pos = i;
+			return text != null ? new Token(text, start, i) : null;
+		}
+		
+		private boolean isTokenChar(char c, boolean isLetter) {
+			return isLetter ? Character.isLetter(c) : !Character.isWhitespace(c);
+		}
+		
+		private boolean isStopWord(String text) {
+			return stopWords != null && stopWords.contains(text);
+		}
+		
+	}
+		
+}

Added: lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/package.html
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/package.html?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/package.html (added)
+++ lucene/java/trunk/contrib/memory/src/java/org/apache/lucene/index/memory/package.html Mon May  2 02:04:07 2005
@@ -0,0 +1,5 @@
+<HTML>
+<BODY>
+High-performance single-document main memory Apache Lucene fulltext search index.
+</BODY>
+</HTML>

Added: lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java (added)
+++ lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java Mon May  2 02:04:07 2005
@@ -0,0 +1,521 @@
+package org.apache.lucene.index.memory;
+
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * 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.
+ */
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.SimpleAnalyzer;
+import org.apache.lucene.analysis.StopAnalyzer;
+import org.apache.lucene.analysis.StopFilter;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.search.HitCollector;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+
+/**
+Verifies that Lucene MemoryIndex and RAMDirectory have the same behaviour,
+returning the same results for any given query.
+Runs a set of queries against a set of files and compares results for identity.
+Can also be used as a simple benchmark.
+<p>
+Example usage:
+<pre>
+cd lucene-cvs
+java org.apache.lucene.index.memory.MemoryIndexTest 1 1 memram @testqueries.txt *.txt *.html *.xml xdocs/*.xml src/test/org/apache/lucene/queryParser/*.java 
+</pre>
+where testqueries.txt is a file with one query per line, such as:
+<pre>
+#
+# queries extracted from TestQueryParser.java
+#
+Apache
+Apach~ AND Copy*
+
+a AND b
+(a AND b)
+c OR (a AND b)
+a AND NOT b
+a AND -b
+a AND !b
+a && b
+a && ! b
+
+a OR b
+a || b
+a OR !b
+a OR ! b
+a OR -b
+
++term -term term
+foo:term AND field:anotherTerm
+term AND "phrase phrase"
+"hello there"
+
+germ term^2.0
+(term)^2.0
+(germ term)^2.0
+term^2.0
+term^2
+"germ term"^2.0
+"term germ"^2
+
+(foo OR bar) AND (baz OR boo)
+((a OR b) AND NOT c) OR d
++(apple "steve jobs") -(foo bar baz)
++title:(dog OR cat) -author:"bob dole"
+
+
+a&b
+a&&b
+.NET
+
+"term germ"~2
+"term germ"~2 flork
+"term"~2
+"~2 germ"
+"term germ"~2^2
+
+3
+term 1.0 1 2
+term term1 term2
+
+term*
+term*^2
+term~
+term~0.7
+term~^2
+term^2~
+term*germ
+term*germ^3
+
+
+term*
+Term*
+TERM*
+term*
+Term*
+TERM*
+
+// Then 'full' wildcard queries:
+te?m
+Te?m
+TE?M
+Te?m*gerM
+te?m
+Te?m
+TE?M
+Te?m*gerM
+
+term term term
+term +stop term
+term -stop term
+drop AND stop AND roll
+term phrase term
+term AND NOT phrase term
+stop
+
+
+[ a TO c]
+[ a TO c ]
+{ a TO c}
+{ a TO c }
+{ a TO c }^2.0
+[ a TO c] OR bar
+[ a TO c] AND bar
+( bar blar { a TO c}) 
+gack ( bar blar { a TO c}) 
+
+
++weltbank +worlbank
++weltbank\n+worlbank
+weltbank \n+worlbank
+weltbank \n +worlbank
++weltbank\r+worlbank
+weltbank \r+worlbank
+weltbank \r +worlbank
++weltbank\r\n+worlbank
+weltbank \r\n+worlbank
+weltbank \r\n +worlbank
+weltbank \r \n +worlbank
++weltbank\t+worlbank
+weltbank \t+worlbank
+weltbank \t +worlbank
+
+
+term term term
+term +term term
+term term +term
+term +term +term
+-term term term
+
+
+on^1.0
+"hello"^2.0
+hello^2.0
+"on"^1.0
+the^3
+</pre>
+
+@author whoschek.AT.lbl.DOT.gov
+*/
+public class MemoryIndexTest extends TestCase {
+	
+	private Analyzer analyzer;
+	private boolean fastMode = false;
+	
+	private static final String FIELD_NAME = "content";
+
+	/** Runs the tests and/or benchmark */
+	public static void main(String[] args) throws Throwable {
+		new MemoryIndexTest().run(args);		
+	}
+
+//	public void setUp() {	}
+//	public void tearDown() {}
+	
+	public void testMany() throws Throwable {
+		String[] files = listFiles(new String[] {
+			"*.txt", "*.html", "*.xml", "xdocs/*.xml", 
+			"src/java/test/org/apache/lucene/queryParser/*.java",
+			"src/java/org/apache/lucene/index/memory/*.java",
+		});
+		System.out.println("files = " + java.util.Arrays.asList(files));
+		String[] xargs = new String[] {
+			"1", "1", "memram", 
+			"@src/test/org/apache/lucene/index/memory/testqueries.txt",
+		};
+		String[] args = new String[xargs.length + files.length];
+		System.arraycopy(xargs, 0, args, 0, xargs.length);
+		System.arraycopy(files, 0, args, xargs.length, files.length);
+		run(args);
+	}
+	
+	private void run(String[] args) throws Throwable {
+		int k = -1;
+		
+		int iters = 1;
+		if (args.length > ++k) iters = Math.max(1, Integer.parseInt(args[k]));
+		
+		int runs = 1;
+		if (args.length > ++k) runs = Math.max(1, Integer.parseInt(args[k]));
+		
+		String cmd = "memram";
+		if (args.length > ++k) cmd = args[k];
+		boolean useMemIndex = cmd.indexOf("mem") >= 0;
+		boolean useRAMIndex = cmd.indexOf("ram") >= 0;
+		
+		String[] queries = { "term", "term*", "term~", "Apache", "Apach~ AND Copy*" };
+		if (args.length > ++k) {
+			String arg = args[k];
+			if (arg.startsWith("@")) 
+				queries = readLines(new File(arg.substring(1)));
+			else
+				queries = new String[] { arg };
+		}
+		
+		File[] files = new File[] {new File("CHANGES.txt"), new File("LICENSE.txt") };
+		if (args.length > ++k) {
+			files = new File[args.length - k];
+			for (int i=k; i < args.length; i++) {
+				files[i-k] = new File(args[i]);
+			}
+		}
+		
+		boolean toLowerCase = true;
+//		boolean toLowerCase = false;
+//		Set stopWords = null;
+		Set stopWords = StopFilter.makeStopSet(StopAnalyzer.ENGLISH_STOP_WORDS);
+		
+		Analyzer[] analyzers = new Analyzer[] { 
+				new SimpleAnalyzer(),
+				new StopAnalyzer(),
+				new StandardAnalyzer(),
+				PatternAnalyzer.DEFAULT_ANALYZER,
+//				new WhitespaceAnalyzer(),
+//				new PatternAnalyzer(PatternAnalyzer.NON_WORD_PATTERN, false, null),
+//				new PatternAnalyzer(PatternAnalyzer.NON_WORD_PATTERN, true, stopWords),				
+//				new SnowballAnalyzer("English", StopAnalyzer.ENGLISH_STOP_WORDS),
+		};
+		
+		for (int iter=0; iter < iters; iter++) {
+			System.out.println("\n########### iteration=" + iter);
+			long start = System.currentTimeMillis();						
+			long bytes = 0;
+			
+			for (int anal=0; anal < analyzers.length; anal++) {
+				this.analyzer = analyzers[anal];
+				
+				for (int i=0; i < files.length; i++) {
+					File file = files[i];
+					if (!file.exists() || file.isDirectory()) continue; // ignore
+					bytes += file.length();
+					String text = toString(new FileInputStream(file), null);
+					Document doc = createDocument(text);
+					System.out.println("\n*********** FILE=" + file);
+					
+					for (int q=0; q < queries.length; q++) {
+						try {
+							Query query = parseQuery(queries[q]);
+							
+							for (int run=0; run < runs; run++) {
+								float score1 = 0.0f; float score2 = 0.0f;
+								if (useMemIndex) score1 = query(createMemoryIndex(doc), query); 
+								if (useRAMIndex) score2 = query(createRAMIndex(doc), query);
+								if (useMemIndex && useRAMIndex) {
+									System.out.println("diff="+ (score1-score2) + ", query=" + queries[q] + ", s1=" + score1 + ", s2=" + score2);
+									if (score1 != score2 || score1 < 0.0f || score2 < 0.0f || score1 > 1.0f || score2 > 1.0f) {
+										throw new IllegalStateException("BUG DETECTED:" + (i*(q+1)) + " at query=" + queries[q] + ", file=" + file + ", anal=" + analyzer);
+									}
+								}
+							}
+						} catch (Throwable t) {
+							if (t instanceof OutOfMemoryError) t.printStackTrace();
+							System.out.println("Fatal error at query=" + queries[q] + ", file=" + file + ", anal=" + analyzer);
+							throw t;
+						}
+					}
+				}
+			}
+			long end = System.currentTimeMillis();
+			System.out.println("\nsecs = " + ((end-start)/1000.0f));
+			System.out.println("queries/sec= " + 
+				(1.0f * runs * queries.length * analyzers.length * files.length 
+						/ ((end-start)/1000.0f)));
+			float mb = (1.0f * bytes * queries.length * runs) / (1024.0f * 1024.0f);
+			System.out.println("MB/sec = " + (mb / ((end-start)/1000.0f)));
+		}
+		
+		if (useMemIndex && useRAMIndex) 
+			System.out.println("No bug found. done.");
+		else 
+			System.out.println("Done benchmarking (without checking correctness).");
+	}
+	
+	// returns file line by line, ignoring empty lines and comments
+	private String[] readLines(File file) throws Exception {
+		BufferedReader reader = new BufferedReader(new InputStreamReader(
+				new FileInputStream(file))); 
+		ArrayList lines = new ArrayList();
+		String line;	
+		while ((line = reader.readLine()) != null) {
+			String t = line.trim(); 
+			if (t.length() > 0 && t.charAt(0) != '#' && (!t.startsWith("//"))) {
+				lines.add(line);
+			}
+		}
+		reader.close();
+		
+		String[] result = new String[lines.size()];
+		lines.toArray(result);
+		return result;
+	}
+	
+	private Document createDocument(String content) {
+		Document doc = new Document();
+		{ // lucene-1.4.3
+			doc.add(Field.UnStored(FIELD_NAME, content));
+		}
+//		{ // lucene >= 1.9
+//			doc.add(new Field(FIELD_NAME, content, Field.Store.NO, Field.Index.TOKENIZED, Field.TermVector.WITH_POSITIONS));
+//		}
+		return doc;
+	}
+	
+	private MemoryIndex createMemoryIndex(Document doc) {
+		MemoryIndex index = new MemoryIndex();
+		Enumeration iter = doc.fields();
+		while (iter.hasMoreElements()) {
+			Field field = (Field) iter.nextElement();
+			index.addField(field.name(), field.stringValue(), analyzer);
+		}
+		return index;
+	}
+	
+	private RAMDirectory createRAMIndex(Document doc) {
+		RAMDirectory dir = new RAMDirectory();		
+		IndexWriter writer = null;
+		try {
+			writer = new IndexWriter(dir, analyzer, true);
+			writer.maxFieldLength = Integer.MAX_VALUE; // ensure large files are scored correctly
+//			writer.setMaxFieldLength(Integer.MAX_VALUE);
+			writer.addDocument(doc);
+			writer.optimize();
+			return dir;
+		} catch (IOException e) { // should never happen (RAMDirectory)
+			throw new RuntimeException(e);
+		} finally {
+			try {
+				if (writer != null) writer.close();
+			} catch (IOException e) { // should never happen (RAMDirectory)
+				throw new RuntimeException(e);
+			}
+		}
+	}
+		
+	private float query(Object index, Query query) {
+//		System.out.println("MB=" + (getMemorySize(index) / (1024.0f * 1024.0f)));
+		Searcher searcher = null;
+		try {
+			if (index instanceof Directory)
+				searcher = new IndexSearcher((Directory)index);
+			else 
+				searcher = ((MemoryIndex) index).createSearcher();
+
+			final float[] scores = new float[1]; // inits to 0.0f
+			searcher.search(query, new HitCollector() {
+				public void collect(int doc, float score) {
+					scores[0] = score;
+				}
+			});
+			float score = scores[0];
+//			Hits hits = searcher.search(query);
+//			float score = hits.length() > 0 ? hits.score(0) : 0.0f;
+			return score;
+		} catch (IOException e) { // should never happen (RAMDirectory)
+			throw new RuntimeException(e);
+		} finally {
+			try {
+				if (searcher != null) searcher.close();
+			} catch (IOException e) { // should never happen (RAMDirectory)
+				throw new RuntimeException(e);
+			}
+		}
+	}
+	
+	private int getMemorySize(Object index) {
+		if (index instanceof Directory) {
+			try {
+				Directory dir = (Directory) index;
+				int size = 0;
+				String[] fileNames = dir.list();
+				for (int i=0; i < fileNames.length; i++) {
+					size += dir.fileLength(fileNames[i]);
+				}
+				return size;
+			}
+			catch (IOException e) { // can never happen (RAMDirectory)
+				throw new RuntimeException(e);
+			}
+		}
+		else {
+			return ((MemoryIndex) index).getMemorySize();
+		}
+	}
+	
+	private Query parseQuery(String expression) throws ParseException {
+		QueryParser parser = new QueryParser(FIELD_NAME, analyzer);
+//		parser.setPhraseSlop(0);
+		return parser.parse(expression);
+	}
+	
+	/** returns all files matching the given file name patterns (quick n'dirty) */
+	static String[] listFiles(String[] fileNames) {
+		LinkedHashSet allFiles = new LinkedHashSet();
+		for (int i=0; i < fileNames.length; i++) {
+			int k;
+			if ((k = fileNames[i].indexOf("*")) < 0) {
+				allFiles.add(fileNames[i]);
+			} else {
+				String prefix = fileNames[i].substring(0, k);
+				if (prefix.length() == 0) prefix = ".";
+				final String suffix = fileNames[i].substring(k+1);
+				File[] files = new File(prefix).listFiles(new FilenameFilter() {
+					public boolean accept(File dir, String name) {
+						return name.endsWith(suffix);
+					}
+				});
+				if (files != null) {
+					for (int j=0; j < files.length; j++) {
+						allFiles.add(files[j].getPath());
+					}
+				}
+			}			
+		}
+		
+		String[] result = new String[allFiles.size()];
+		allFiles.toArray(result);
+		return result;
+	}
+	
+	// trick to detect default platform charset
+	private static final Charset DEFAULT_PLATFORM_CHARSET = 
+		Charset.forName(new InputStreamReader(new ByteArrayInputStream(new byte[0])).getEncoding());	
+	
+	// the following utility methods below are copied from Apache style Nux library - see http://dsd.lbl.gov/nux
+	private static String toString(InputStream input, Charset charset) throws IOException {
+		if (charset == null) charset = DEFAULT_PLATFORM_CHARSET;			
+		byte[] data = toByteArray(input);
+		return charset.decode(ByteBuffer.wrap(data)).toString();
+	}
+	
+	private static byte[] toByteArray(InputStream input) throws IOException {
+		try {
+			// safe and fast even if input.available() behaves weird or buggy
+			int len = Math.max(256, input.available());
+			byte[] buffer = new byte[len];
+			byte[] output = new byte[len];
+			
+			len = 0;
+			int n;
+			while ((n = input.read(buffer)) >= 0) {
+				if (len + n > output.length) { // grow capacity
+					byte tmp[] = new byte[Math.max(output.length << 1, len + n)];
+					System.arraycopy(output, 0, tmp, 0, len);
+					System.arraycopy(buffer, 0, tmp, len, n);
+					buffer = output; // use larger buffer for future larger bulk reads
+					output = tmp;
+				} else {
+					System.arraycopy(buffer, 0, output, len, n);
+				}
+				len += n;
+			}
+
+			if (len == output.length) return output;
+			buffer = null; // help gc
+			buffer = new byte[len];
+			System.arraycopy(output, 0, buffer, 0, len);
+			return buffer;
+		} finally {
+			if (input != null) input.close();
+		}
+	}
+	
+}
\ No newline at end of file

Added: lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/PatternAnalyzerTest.java
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/PatternAnalyzerTest.java?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/PatternAnalyzerTest.java (added)
+++ lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/PatternAnalyzerTest.java Mon May  2 02:04:07 2005
@@ -0,0 +1,279 @@
+package org.apache.lucene.index.memory;
+
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * 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.
+ */
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.StringReader;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.LetterTokenizer;
+import org.apache.lucene.analysis.LowerCaseFilter;
+import org.apache.lucene.analysis.StopAnalyzer;
+import org.apache.lucene.analysis.StopFilter;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.WhitespaceTokenizer;
+
+/**
+Verifies that Lucene PatternAnalyzer and normal Lucene Analyzers have the same behaviour,
+returning the same results for any given free text.
+Runs a set of texts against a tokenizers/analyzers
+Can also be used as a simple benchmark.
+<p>
+Example usage:
+<pre>
+cd lucene-cvs
+java org.apache.lucene.index.memory.PatternAnalyzerTest 1 1 patluc 1 2 2 *.txt *.xml docs/*.html src/java/org/apache/lucene/index/*.java xdocs/*.xml ../nux/samples/data/*.xml
+</pre>
+
+with WhitespaceAnalyzer problems can be found; These are not bugs but questionable 
+Lucene features: CharTokenizer.MAX_WORD_LEN = 255.
+Thus the PatternAnalyzer produces correct output, whereas the WhitespaceAnalyzer 
+silently truncates text, and so the comparison results in assertEquals() don't match up. 
+
+@author whoschek.AT.lbl.DOT.gov
+*/
+public class PatternAnalyzerTest extends TestCase {
+	
+	/** Runs the tests and/or benchmark */
+	public static void main(String[] args) throws Throwable {
+		new PatternAnalyzerTest().run(args);		
+	}
+	
+	public void testMany() throws Throwable {
+		String[] files = MemoryIndexTest.listFiles(new String[] {
+			"*.txt", "*.html", "*.xml", "xdocs/*.xml", 
+			"src/test/org/apache/lucene/queryParser/*.java",
+			"src/org/apache/lucene/index/memory/*.java",
+		});
+		System.out.println("files = " + java.util.Arrays.asList(files));
+		String[] xargs = new String[] {
+			"1", "1", "patluc", "1", "2", "2",
+		};
+		String[] args = new String[xargs.length + files.length];
+		System.arraycopy(xargs, 0, args, 0, xargs.length);
+		System.arraycopy(files, 0, args, xargs.length, files.length);
+		run(args);
+	}
+	
+	private void run(String[] args) throws Throwable {
+		int k = -1;
+		
+		int iters = 1;
+		if (args.length > ++k) iters = Math.max(1, Integer.parseInt(args[k]));
+		
+		int runs = 1;
+		if (args.length > ++k) runs = Math.max(1, Integer.parseInt(args[k]));
+		
+		String cmd = "patluc";
+		if (args.length > ++k) cmd = args[k];
+		boolean usePattern = cmd.indexOf("pat") >= 0;
+		boolean useLucene  = cmd.indexOf("luc") >= 0;
+		
+		int maxLetters = 1; // = 2: CharTokenizer.MAX_WORD_LEN issue; see class javadoc
+		if (args.length > ++k) maxLetters = Integer.parseInt(args[k]);
+		
+		int maxToLower = 2;
+		if (args.length > ++k) maxToLower = Integer.parseInt(args[k]);
+
+		int maxStops = 2;
+		if (args.length > ++k) maxStops = Integer.parseInt(args[k]);
+		
+		File[] files = new File[] {new File("CHANGES.txt"), new File("LICENSE.txt") };
+		if (args.length > ++k) {
+			files = new File[args.length - k];
+			for (int i=k; i < args.length; i++) {
+				files[i-k] = new File(args[i]);
+			}
+		}
+		
+		for (int iter=0; iter < iters; iter++) {
+			System.out.println("\n########### iteration=" + iter);
+			long start = System.currentTimeMillis();						
+			long bytes = 0;
+			
+			for (int i=0; i < files.length; i++) {
+				File file = files[i];
+				if (!file.exists() || file.isDirectory()) continue; // ignore
+				bytes += file.length();
+				String text = toString(new FileInputStream(file), null);
+				System.out.println("\n*********** FILE=" + file);
+
+				for (int letters=0; letters < maxLetters; letters++) {
+					boolean lettersOnly = letters == 0;
+					
+					for (int stops=0; stops < maxStops; stops++) {
+						Set stopWords = null;
+						if (stops != 0) stopWords = StopFilter.makeStopSet(StopAnalyzer.ENGLISH_STOP_WORDS);
+								
+						for (int toLower=0; toLower < maxToLower; toLower++) {
+							boolean toLowerCase = toLower != 0;
+								
+							for (int run=0; run < runs; run++) {
+								List tokens1 = null; List tokens2 = null;
+								try {
+									if (usePattern) tokens1 = getTokens(patternTokenStream(text, lettersOnly, toLowerCase, stopWords));
+									if (useLucene) tokens2 = getTokens(luceneTokenStream(text, lettersOnly, toLowerCase, stopWords));					
+									if (usePattern && useLucene) assertEquals(tokens1, tokens2);
+								} catch (Throwable t) {
+									if (t instanceof OutOfMemoryError) t.printStackTrace();
+									System.out.println("fatal error at file=" + file + ", letters="+ lettersOnly + ", toLowerCase=" + toLowerCase + ", stopwords=" + (stopWords != null ? "english" : "none"));
+									System.out.println("\n\ntokens1=" + toString(tokens1));
+									System.out.println("\n\ntokens2=" + toString(tokens2));
+									throw t;
+								}
+							}
+						}
+					}
+				}
+				long end = System.currentTimeMillis();
+				System.out.println("\nsecs = " + ((end-start)/1000.0f));
+				System.out.println("files/sec= " + 
+						(1.0f * runs * maxLetters * maxToLower * maxStops * files.length 
+						/ ((end-start)/1000.0f)));
+				float mb = (1.0f * bytes * runs * maxLetters * maxToLower * maxStops) / (1024.0f * 1024.0f);
+				System.out.println("MB/sec = " + (mb / ((end-start)/1000.0f)));
+			}
+		}
+		
+		if (usePattern && useLucene) 
+			System.out.println("No bug found. done.");
+		else 
+			System.out.println("Done benchmarking (without checking correctness).");
+	}
+
+	private TokenStream patternTokenStream(String text, boolean letters, boolean toLowerCase, Set stopWords) {
+		Pattern pattern;
+		if (letters) 
+			pattern = PatternAnalyzer.NON_WORD_PATTERN;
+		else 							
+			pattern = PatternAnalyzer.WHITESPACE_PATTERN;
+		PatternAnalyzer analyzer = new PatternAnalyzer(pattern, toLowerCase, stopWords);
+		return analyzer.tokenStream("", text);
+	}
+	
+	private TokenStream luceneTokenStream(String text, boolean letters, boolean toLowerCase, Set stopWords) {
+		TokenStream stream;
+		if (letters) 
+			stream = new LetterTokenizer(new StringReader(text));
+		else
+			stream = new WhitespaceTokenizer(new StringReader(text));
+		if (toLowerCase)	stream = new LowerCaseFilter(stream);
+		if (stopWords != null) stream = new StopFilter(stream, stopWords);
+		return stream;						
+	}
+	
+	private List getTokens(TokenStream stream) throws IOException {
+		ArrayList tokens = new ArrayList();
+		Token token;
+		while ((token = stream.next()) != null) {
+			tokens.add(token);
+		}
+		return tokens;
+	}
+	
+	private void assertEquals(List tokens1, List tokens2) {
+		int size = Math.min(tokens1.size(), tokens2.size());
+		int i=0;
+		try {
+			for (; i < size; i++) {
+				Token t1 = (Token) tokens1.get(i);
+				Token t2 = (Token) tokens2.get(i);
+				if (!(t1.termText().equals(t2.termText()))) throw new IllegalStateException("termText");
+				if (t1.startOffset() != t2.startOffset()) throw new IllegalStateException("startOffset");
+				if (t1.endOffset() != t2.endOffset()) throw new IllegalStateException("endOffset");
+				if (!(t1.type().equals(t2.type()))) throw new IllegalStateException("type");
+			}
+			if (tokens1.size() != tokens2.size()) 	throw new IllegalStateException("size1=" + tokens1.size() + ", size2=" + tokens2.size());
+		}
+
+		catch (IllegalStateException e) {
+			if (size > 0) {
+				System.out.println("i=" + i + ", size=" + size);
+				System.out.println("t1[size]='" + ((Token) tokens1.get(size-1)).termText() + "'");
+				System.out.println("t2[size]='" + ((Token) tokens2.get(size-1)).termText() + "'");
+			}
+			throw e;
+		}
+	}
+	
+	private String toString(List tokens) {
+		if (tokens == null) return "null";
+		String str = "[";
+		for (int i=0; i < tokens.size(); i++) {
+			Token t1 = (Token) tokens.get(i);
+			str = str + "'" + t1.termText() + "', ";
+		}
+		return str + "]";
+	}
+	
+	// trick to detect default platform charset
+	private static final Charset DEFAULT_PLATFORM_CHARSET = 
+		Charset.forName(new InputStreamReader(new ByteArrayInputStream(new byte[0])).getEncoding());	
+	
+	// the following utility methods below are copied from Apache style Nux library - see http://dsd.lbl.gov/nux
+	private static String toString(InputStream input, Charset charset) throws IOException {
+		if (charset == null) charset = DEFAULT_PLATFORM_CHARSET;			
+		byte[] data = toByteArray(input);
+		return charset.decode(ByteBuffer.wrap(data)).toString();
+	}
+	
+	private static byte[] toByteArray(InputStream input) throws IOException {
+		try {
+			// safe and fast even if input.available() behaves weird or buggy
+			int len = Math.max(256, input.available());
+			byte[] buffer = new byte[len];
+			byte[] output = new byte[len];
+			
+			len = 0;
+			int n;
+			while ((n = input.read(buffer)) >= 0) {
+				if (len + n > output.length) { // grow capacity
+					byte tmp[] = new byte[Math.max(output.length << 1, len + n)];
+					System.arraycopy(output, 0, tmp, 0, len);
+					System.arraycopy(buffer, 0, tmp, len, n);
+					buffer = output; // use larger buffer for future larger bulk reads
+					output = tmp;
+				} else {
+					System.arraycopy(buffer, 0, output, len, n);
+				}
+				len += n;
+			}
+
+			if (len == output.length) return output;
+			buffer = null; // help gc
+			buffer = new byte[len];
+			System.arraycopy(output, 0, buffer, 0, len);
+			return buffer;
+		} finally {
+			if (input != null) input.close();
+		}
+	}
+	
+}
\ No newline at end of file

Added: lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/testqueries.txt
URL: http://svn.apache.org/viewcvs/lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/testqueries.txt?rev=165606&view=auto
==============================================================================
--- lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/testqueries.txt (added)
+++ lucene/java/trunk/contrib/memory/src/test/org/apache/lucene/index/memory/testqueries.txt Mon May  2 02:04:07 2005
@@ -0,0 +1,129 @@
+#
+# queries extracted from TestQueryParser.java
+#
+Apache
+Apach~ AND Copy*
+
+a AND b
+(a AND b)
+c OR (a AND b)
+a AND NOT b
+a AND -b
+a AND !b
+a && b
+a && ! b
+
+a OR b
+a || b
+a OR !b
+a OR ! b
+a OR -b
+
++term -term term
+foo:term AND field:anotherTerm
+term AND "phrase phrase"
+"hello there"
+
+germ term^2.0
+(term)^2.0
+(germ term)^2.0
+term^2.0
+term^2
+"germ term"^2.0
+"term germ"^2
+
+(foo OR bar) AND (baz OR boo)
+((a OR b) AND NOT c) OR d
++(apple "steve jobs") -(foo bar baz)
++title:(dog OR cat) -author:"bob dole"
+
+
+a&b
+a&&b
+.NET
+
+"term germ"~2
+"term germ"~2 flork
+"term"~2
+"~2 germ"
+"term germ"~2^2
+
+3
+term 1.0 1 2
+term term1 term2
+
+term*
+term*^2
+term~
+term~0.7
+term~^2
+term^2~
+term*germ
+term*germ^3
+
+
+term*
+Term*
+TERM*
+term*
+Term*
+TERM*
+
+// Then 'full' wildcard queries:
+te?m
+Te?m
+TE?M
+Te?m*gerM
+te?m
+Te?m
+TE?M
+Te?m*gerM
+
+term term term
+term +stop term
+term -stop term
+drop AND stop AND roll
+term phrase term
+term AND NOT phrase term
+stop
+
+
+[ a TO c]
+[ a TO c ]
+{ a TO c}
+{ a TO c }
+{ a TO c }^2.0
+[ a TO c] OR bar
+[ a TO c] AND bar
+( bar blar { a TO c}) 
+gack ( bar blar { a TO c}) 
+
+
++weltbank +worlbank
++weltbank\n+worlbank
+weltbank \n+worlbank
+weltbank \n +worlbank
++weltbank\r+worlbank
+weltbank \r+worlbank
+weltbank \r +worlbank
++weltbank\r\n+worlbank
+weltbank \r\n+worlbank
+weltbank \r\n +worlbank
+weltbank \r \n +worlbank
++weltbank\t+worlbank
+weltbank \t+worlbank
+weltbank \t +worlbank
+
+
+term term term
+term +term term
+term term +term
+term +term +term
+-term term term
+
+
+on^1.0
+"hello"^2.0
+hello^2.0
+"on"^1.0
+the^3