Return-Path: X-Original-To: apmail-lucene-commits-archive@www.apache.org Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 12FC678DD for ; Fri, 29 Jul 2011 00:13:31 +0000 (UTC) Received: (qmail 87647 invoked by uid 500); 29 Jul 2011 00:13:30 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 87640 invoked by uid 99); 29 Jul 2011 00:13:30 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jul 2011 00:13:30 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jul 2011 00:13:23 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 765A3238885D; Fri, 29 Jul 2011 00:13:01 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1152055 - in /lucene/dev/trunk/lucene/src: java/org/apache/lucene/index/ java/org/apache/lucene/store/ test/org/apache/lucene/index/ Date: Fri, 29 Jul 2011 00:13:00 -0000 To: commits@lucene.apache.org From: rmuir@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110729001301.765A3238885D@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: rmuir Date: Fri Jul 29 00:12:54 2011 New Revision: 1152055 URL: http://svn.apache.org/viewvc?rev=1152055&view=rev Log: LUCENE-3342: improve FrozenBufferedDeletes ram efficiency Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CoalescedDeletes.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java (with props) lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (with props) Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMFile.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java Fri Jul 29 00:12:54 2011 @@ -18,17 +18,14 @@ package org.apache.lucene.index; */ import java.util.ArrayList; -import java.util.Iterator; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.search.Query; import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; /* Holds buffered deletes, by docID, term or query for a * single segment. This is used to hold buffered pending @@ -67,7 +64,7 @@ class BufferedDeletes { final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24; final AtomicInteger numTermDeletes = new AtomicInteger(); - final Map terms; + final Map terms = new HashMap(); final Map queries = new HashMap(); final List docIDs = new ArrayList(); @@ -78,18 +75,13 @@ class BufferedDeletes { private final static boolean VERBOSE_DELETES = false; long gen; - public BufferedDeletes(boolean sortTerms) { - this(sortTerms, new AtomicLong()); + public BufferedDeletes() { + this(new AtomicLong()); } - BufferedDeletes(boolean sortTerms, AtomicLong bytesUsed) { + BufferedDeletes(AtomicLong bytesUsed) { assert bytesUsed != null; this.bytesUsed = bytesUsed; - if (sortTerms) { - terms = new TreeMap(); - } else { - terms = new HashMap(); - } } @Override @@ -117,50 +109,6 @@ class BufferedDeletes { } } - void update(BufferedDeletes in) { - numTermDeletes.addAndGet(in.numTermDeletes.get()); - for (Map.Entry ent : in.terms.entrySet()) { - final Term term = ent.getKey(); - if (!terms.containsKey(term)) { - // only incr bytesUsed if this term wasn't already buffered: - bytesUsed.addAndGet(BYTES_PER_DEL_TERM); - } - terms.put(term, MAX_INT); - } - - for (Map.Entry ent : in.queries.entrySet()) { - final Query query = ent.getKey(); - if (!queries.containsKey(query)) { - // only incr bytesUsed if this query wasn't already buffered: - bytesUsed.addAndGet(BYTES_PER_DEL_QUERY); - } - queries.put(query, MAX_INT); - } - - // docIDs never move across segments and the docIDs - // should already be cleared - } - - void update(FrozenBufferedDeletes in) { - numTermDeletes.addAndGet(in.numTermDeletes); - for(Term term : in.terms) { - if (!terms.containsKey(term)) { - // only incr bytesUsed if this term wasn't already buffered: - bytesUsed.addAndGet(BYTES_PER_DEL_TERM); - } - terms.put(term, MAX_INT); - } - - for(int queryIdx=0;queryIdx termsIterable() { - return new Iterable() { - @Override - public Iterator iterator() { - return terms.keySet().iterator(); - } - }; - } - - public Iterable queriesIterable() { - return new Iterable() { - - @Override - public Iterator iterator() { - return new Iterator() { - private final Iterator> iter = queries.entrySet().iterator(); - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public QueryAndLimit next() { - final Map.Entry ent = iter.next(); - return new QueryAndLimit(ent.getKey(), ent.getValue()); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - }; - } - + void clear() { terms.clear(); queries.clear(); Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Fri Jul 29 00:12:54 2011 @@ -33,6 +33,7 @@ import org.apache.lucene.search.DocIdSet import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryWrapperFilter; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.BytesRef; /* Tracks the stream of {@link BufferedDeletes}. * When DocumentsWriterPerThread flushes, its buffered @@ -182,7 +183,7 @@ class BufferedDeletesStream { infos2.addAll(infos); Collections.sort(infos2, sortSegInfoByDelGen); - BufferedDeletes coalescedDeletes = null; + CoalescedDeletes coalescedDeletes = null; boolean anyNewDeletes = false; int infosIDX = infos2.size()-1; @@ -200,7 +201,7 @@ class BufferedDeletesStream { if (packet != null && segGen < packet.delGen()) { //System.out.println(" coalesce"); if (coalescedDeletes == null) { - coalescedDeletes = new BufferedDeletes(true); + coalescedDeletes = new CoalescedDeletes(); } if (!packet.isSegmentPrivate) { /* @@ -251,7 +252,7 @@ class BufferedDeletesStream { } if (coalescedDeletes == null) { - coalescedDeletes = new BufferedDeletes(true); + coalescedDeletes = new CoalescedDeletes(); } /* @@ -463,7 +464,8 @@ class BufferedDeletesStream { if (term != null) { assert lastDeleteTerm == null || term.compareTo(lastDeleteTerm) > 0: "lastTerm=" + lastDeleteTerm + " vs term=" + term; } - lastDeleteTerm = term; + // TODO: we re-use term now in our merged iterable, but we shouldn't clone, instead copy for this assert + lastDeleteTerm = term == null ? null : new Term(term.field(), new BytesRef(term.bytes)); return true; } Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CoalescedDeletes.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CoalescedDeletes.java?rev=1152055&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CoalescedDeletes.java (added) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CoalescedDeletes.java Fri Jul 29 00:12:54 2011 @@ -0,0 +1,191 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.search.Query; +import org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; + +class CoalescedDeletes { + final Map queries = new HashMap(); + final List> iterables = new ArrayList>(); + + @Override + public String toString() { + // note: we could add/collect more debugging information + return "CoalescedDeletes(termSets=" + iterables.size() + ",queries=" + queries.size() + ")"; + } + + void update(FrozenBufferedDeletes in) { + iterables.add(in.termsIterable()); + + for(int queryIdx=0;queryIdx termsIterable() { + return new Iterable() { + @Override + public Iterator iterator() { + ArrayList> subs = new ArrayList>(iterables.size()); + for (Iterable iterable : iterables) { + subs.add(iterable.iterator()); + } + return mergedIterator(subs); + } + }; + } + + public Iterable queriesIterable() { + return new Iterable() { + + @Override + public Iterator iterator() { + return new Iterator() { + private final Iterator> iter = queries.entrySet().iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public QueryAndLimit next() { + final Map.Entry ent = iter.next(); + return new QueryAndLimit(ent.getKey(), ent.getValue()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; + } + + /** provides a merged view across multiple iterators */ + static Iterator mergedIterator(final List> iterators) { + return new Iterator() { + Term current; + TermMergeQueue queue = new TermMergeQueue(iterators.size()); + SubIterator[] top = new SubIterator[iterators.size()]; + int numTop; + + { + int index = 0; + for (Iterator iterator : iterators) { + if (iterator.hasNext()) { + SubIterator sub = new SubIterator(); + sub.current = iterator.next(); + sub.iterator = iterator; + sub.index = index++; + queue.add(sub); + } + } + } + + public boolean hasNext() { + if (queue.size() > 0) { + return true; + } + + for (int i = 0; i < numTop; i++) { + if (top[i].iterator.hasNext()) { + return true; + } + } + return false; + } + + public Term next() { + // restore queue + pushTop(); + + // gather equal top fields + if (queue.size() > 0) { + pullTop(); + } else { + current = null; + } + return current; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + + private void pullTop() { + // extract all subs from the queue that have the same top term + assert numTop == 0; + while (true) { + top[numTop++] = queue.pop(); + if (queue.size() == 0 + || !(queue.top()).current.equals(top[0].current)) { + break; + } + } + current = top[0].current; + } + + private void pushTop() { + // call next() on each top, and put back into queue + for (int i = 0; i < numTop; i++) { + if (top[i].iterator.hasNext()) { + top[i].current = top[i].iterator.next(); + queue.add(top[i]); + } else { + // no more terms + top[i].current = null; + } + } + numTop = 0; + } + }; + } + + private static class SubIterator { + Iterator iterator; + Term current; + int index; + } + + private static class TermMergeQueue extends PriorityQueue { + TermMergeQueue(int size) { + super(size); + } + + @Override + protected boolean lessThan(SubIterator a, SubIterator b) { + final int cmp = a.current.compareTo(b.current); + if (cmp != 0) { + return cmp < 0; + } else { + return a.index < b.index; + } + } + } +} Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java Fri Jul 29 00:12:54 2011 @@ -81,7 +81,7 @@ final class DocumentsWriterDeleteQueue { } DocumentsWriterDeleteQueue(long generation) { - this(new BufferedDeletes(false), generation); + this(new BufferedDeletes(), generation); } DocumentsWriterDeleteQueue(BufferedDeletes globalBufferedDeletes, long generation) { Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Fri Jul 29 00:12:54 2011 @@ -188,7 +188,7 @@ public class DocumentsWriterPerThread { bytesUsed = new AtomicLong(0); byteBlockAllocator = new DirectTrackingAllocator(bytesUsed); consumer = indexingChain.getChain(this); - pendingDeletes = new BufferedDeletes(false); + pendingDeletes = new BufferedDeletes(); initialize(); } @@ -476,7 +476,7 @@ public class DocumentsWriterPerThread { segmentDeletes = null; } else { segmentDeletes = pendingDeletes; - pendingDeletes = new BufferedDeletes(false); + pendingDeletes = new BufferedDeletes(); } if (infoStream != null) { Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FrozenBufferedDeletes.java Fri Jul 29 00:12:54 2011 @@ -21,31 +21,23 @@ import java.util.Iterator; import java.util.Map; import org.apache.lucene.search.Query; +import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit; /** Holds buffered deletes by term or query, once pushed. - * Pushed deltes are write-once, so we shift to more + * Pushed deletes are write-once, so we shift to more * memory efficient data structure to hold them. We don't * hold docIDs because these are applied on flush. */ class FrozenBufferedDeletes { - /* Rough logic: Term is object w/ - String field and BytesRef text (OBJ_HEADER + 2*POINTER). - String field is (OBJ_HEADER + 4*INT + - POINTER + OBJ_HEADER + CHAR*field.length). - Term's text is BytesRef (OBJ_HEADER + 2*INT + POINTER + - OBJ_HEADER + bytes.length). */ - final static int BYTES_PER_DEL_TERM = 4*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 5*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 6*RamUsageEstimator.NUM_BYTES_INT; - /* Query we often undercount (say 24 bytes), plus int. */ final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_INT + 24; // Terms, in sorted order: - // TODO: would be more RAM efficient to store BytesRef[], - // per field: - final Term[] terms; + final PrefixCodedTerms terms; + int termCount; // just for debugging // Parallel array of deleted query, and the docIDUpto for // each @@ -62,7 +54,15 @@ class FrozenBufferedDeletes { public FrozenBufferedDeletes(BufferedDeletes deletes, boolean isSegmentPrivate) { this.isSegmentPrivate = isSegmentPrivate; assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries"; - terms = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]); + Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]); + termCount = termsArray.length; + ArrayUtil.mergeSort(termsArray); + PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder(); + for (Term term : termsArray) { + builder.add(term); + } + terms = builder.finish(); + queries = new Query[deletes.queries.size()]; queryLimits = new int[deletes.queries.size()]; int upto = 0; @@ -71,13 +71,8 @@ class FrozenBufferedDeletes { queryLimits[upto] = ent.getValue(); upto++; } - int termDataBytes = 0; - for(Map.Entry ent : deletes.terms.entrySet()) { - final Term term = ent.getKey(); - termDataBytes += term.bytes().length; - termDataBytes += term.field().length() * RamUsageEstimator.NUM_BYTES_CHAR; - } - bytesUsed = terms.length * BYTES_PER_DEL_TERM + queries.length * BYTES_PER_DEL_QUERY + termDataBytes; + + bytesUsed = (int) terms.getSizeInBytes() + queries.length * BYTES_PER_DEL_QUERY; numTermDeletes = deletes.numTermDeletes.get(); } @@ -95,24 +90,7 @@ class FrozenBufferedDeletes { return new Iterable() { @Override public Iterator iterator() { - return new Iterator() { - private int upto; - - @Override - public boolean hasNext() { - return upto < terms.length; - } - - @Override - public Term next() { - return terms[upto++]; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; + return terms.iterator(); } }; } @@ -149,7 +127,7 @@ class FrozenBufferedDeletes { public String toString() { String s = ""; if (numTermDeletes != 0) { - s += " " + numTermDeletes + " deleted terms (unique count=" + terms.length + ")"; + s += " " + numTermDeletes + " deleted terms (unique count=" + termCount + ")"; } if (queries.length != 0) { s += " " + queries.length + " deleted queries"; @@ -162,6 +140,6 @@ class FrozenBufferedDeletes { } boolean any() { - return terms.length > 0 || queries.length > 0; + return termCount > 0 || queries.length > 0; } } Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java?rev=1152055&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java (added) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PrefixCodedTerms.java Fri Jul 29 00:12:54 2011 @@ -0,0 +1,146 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.lucene.index.Term; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RAMFile; +import org.apache.lucene.store.RAMInputStream; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.BytesRef; + +/** + * Prefix codes term instances (prefixes are shared) + * @lucene.experimental + */ +class PrefixCodedTerms implements Iterable { + final RAMFile buffer; + + private PrefixCodedTerms(RAMFile buffer) { + this.buffer = buffer; + } + + /** @return size in bytes */ + public long getSizeInBytes() { + return buffer.getSizeInBytes(); + } + + /** @return iterator over the bytes */ + public Iterator iterator() { + return new PrefixCodedTermsIterator(); + } + + class PrefixCodedTermsIterator implements Iterator { + final IndexInput input; + String field = ""; + BytesRef bytes = new BytesRef(); + Term term = new Term(field, bytes); + + PrefixCodedTermsIterator() { + try { + input = new RAMInputStream(buffer); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public boolean hasNext() { + return input.getFilePointer() < input.length(); + } + + public Term next() { + assert hasNext(); + try { + int code = input.readVInt(); + if ((code & 1) != 0) { + // new field + field = input.readString(); + } + int prefix = code >>> 1; + int suffix = input.readVInt(); + bytes.grow(prefix + suffix); + input.readBytes(bytes.bytes, prefix, suffix); + bytes.length = prefix + suffix; + term.set(field, bytes); + return term; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void remove() { + throw new UnsupportedOperationException(); + } + } + + /** Builds a PrefixCodedTerms: call add repeatedly, then finish. */ + public static class Builder { + private RAMFile buffer = new RAMFile(); + private RAMOutputStream output = new RAMOutputStream(buffer); + private Term lastTerm = new Term(""); + + /** add a term */ + public void add(Term term) { + assert lastTerm.equals(new Term("")) || term.compareTo(lastTerm) > 0; + + try { + int prefix = sharedPrefix(lastTerm.bytes, term.bytes); + int suffix = term.bytes.length - prefix; + if (term.field.equals(lastTerm.field)) { + output.writeVInt(prefix << 1); + } else { + output.writeVInt(prefix << 1 | 1); + output.writeString(term.field); + } + output.writeVInt(suffix); + output.writeBytes(term.bytes.bytes, term.bytes.offset + prefix, suffix); + lastTerm.bytes.copy(term.bytes); + lastTerm.field = term.field; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** return finalized form */ + public PrefixCodedTerms finish() { + try { + output.close(); + return new PrefixCodedTerms(buffer); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private int sharedPrefix(BytesRef term1, BytesRef term2) { + int pos1 = 0; + int pos1End = pos1 + Math.min(term1.length, term2.length); + int pos2 = 0; + while(pos1 < pos1End) { + if (term1.bytes[term1.offset + pos1] != term2.bytes[term2.offset + pos2]) { + return pos1; + } + pos1++; + pos2++; + } + return pos1; + } + } +} Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMFile.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMFile.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMFile.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMFile.java Fri Jul 29 00:12:54 2011 @@ -29,7 +29,7 @@ public class RAMFile { private long lastModified = System.currentTimeMillis(); // File used as buffer, in no RAMDirectory - protected RAMFile() {} + public RAMFile() {} RAMFile(RAMDirectory directory) { this.directory = directory; Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/RAMInputStream.java Fri Jul 29 00:12:54 2011 @@ -19,8 +19,10 @@ package org.apache.lucene.store; import java.io.IOException; -/** A memory-resident {@link IndexInput} implementation. */ -class RAMInputStream extends IndexInput implements Cloneable { +/** A memory-resident {@link IndexInput} implementation. + * + * @lucene.internal */ +public class RAMInputStream extends IndexInput implements Cloneable { static final int BUFFER_SIZE = RAMOutputStream.BUFFER_SIZE; private RAMFile file; @@ -33,7 +35,7 @@ class RAMInputStream extends IndexInput private long bufferStart; private int bufferLength; - RAMInputStream(RAMFile f) throws IOException { + public RAMInputStream(RAMFile f) throws IOException { file = f; length = file.length; if (length/BUFFER_SIZE >= Integer.MAX_VALUE) { Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (original) +++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java Fri Jul 29 00:12:54 2011 @@ -41,8 +41,8 @@ public class TestDocumentsWriterDeleteQu } DeleteSlice slice1 = queue.newSlice(); DeleteSlice slice2 = queue.newSlice(); - BufferedDeletes bd1 = new BufferedDeletes(false); - BufferedDeletes bd2 = new BufferedDeletes(false); + BufferedDeletes bd1 = new BufferedDeletes(); + BufferedDeletes bd2 = new BufferedDeletes(); int last1 = 0; int last2 = 0; Set uniqueValues = new HashSet(); @@ -70,8 +70,11 @@ public class TestDocumentsWriterDeleteQu } assertEquals(uniqueValues, bd1.terms.keySet()); assertEquals(uniqueValues, bd2.terms.keySet()); - assertEquals(uniqueValues, new HashSet(Arrays.asList(queue - .freezeGlobalBuffer(null).terms))); + HashSet frozenSet = new HashSet(); + for (Term t : queue.freezeGlobalBuffer(null).termsIterable()) { + frozenSet.add(t); + } + assertEquals(uniqueValues, frozenSet); assertEquals("num deletes must be 0 after freeze", 0, queue .numGlobalTermDeletes()); } @@ -129,7 +132,7 @@ public class TestDocumentsWriterDeleteQu if (random.nextInt(5) == 0) { FrozenBufferedDeletes freezeGlobalBuffer = queue .freezeGlobalBuffer(null); - assertEquals(termsSinceFreeze, freezeGlobalBuffer.terms.length); + assertEquals(termsSinceFreeze, freezeGlobalBuffer.termCount); assertEquals(queriesSinceFreeze, freezeGlobalBuffer.queries.length); queriesSinceFreeze = 0; termsSinceFreeze = 0; @@ -168,8 +171,11 @@ public class TestDocumentsWriterDeleteQu assertEquals(uniqueValues, deletes.terms.keySet()); } queue.tryApplyGlobalSlice(); - assertEquals(uniqueValues, new HashSet(Arrays.asList(queue - .freezeGlobalBuffer(null).terms))); + HashSet frozenSet = new HashSet(); + for (Term t : queue.freezeGlobalBuffer(null).termsIterable()) { + frozenSet.add(t); + } + assertEquals(uniqueValues, frozenSet); assertEquals("num deletes must be 0 after freeze", 0, queue .numGlobalTermDeletes()); } @@ -188,7 +194,7 @@ public class TestDocumentsWriterDeleteQu this.index = index; this.ids = ids; this.slice = queue.newSlice(); - deletes = new BufferedDeletes(false); + deletes = new BufferedDeletes(); this.latch = latch; } Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1152055&r1=1152054&r2=1152055&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original) +++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Fri Jul 29 00:12:54 2011 @@ -954,9 +954,10 @@ public class TestIndexWriterDelete exten Directory dir = newDirectory(); // Cannot use RandomIndexWriter because we don't want to // ever call commit() for this test: + // note: tiny rambuffer used, as with a 1MB buffer the test is too slow (flush @ 128,999) IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)) - .setRAMBufferSizeMB(1.0f).setMaxBufferedDocs(1000).setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES).setReaderPooling(false)); + .setRAMBufferSizeMB(0.2f).setMaxBufferedDocs(1000).setMergePolicy(NoMergePolicy.NO_COMPOUND_FILES).setReaderPooling(false)); w.setInfoStream(VERBOSE ? System.out : null); int count = 0; while(true) { @@ -981,7 +982,7 @@ public class TestIndexWriterDelete exten } count++; - // Today we applyDelets @ count=7199; even if we make + // Today we applyDeletes @ count=21553; even if we make // sizable improvements to RAM efficiency of buffered // del term we're unlikely to go over 100K: if (count > 100000) { Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java?rev=1152055&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (added) +++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java Fri Jul 29 00:12:54 2011 @@ -0,0 +1,136 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; + +public class TestPrefixCodedTerms extends LuceneTestCase { + + public void testEmpty() { + PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder(); + PrefixCodedTerms pb = b.finish(); + assertFalse(pb.iterator().hasNext()); + } + + public void testOne() { + Term term = new Term("foo", "bogus"); + PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder(); + b.add(term); + PrefixCodedTerms pb = b.finish(); + Iterator iterator = pb.iterator(); + assertTrue(iterator.hasNext()); + assertEquals(term, iterator.next()); + } + + public void testRandom() { + Set terms = new TreeSet(); + int nterms = atLeast(10000); + for (int i = 0; i < nterms; i++) { + Term term = new Term(_TestUtil.randomUnicodeString(random, 2), _TestUtil.randomUnicodeString(random)); + terms.add(term); + } + + PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder(); + for (Term ref: terms) { + b.add(ref); + } + PrefixCodedTerms pb = b.finish(); + + Iterator expected = terms.iterator(); + for (Term t : pb) { + assertTrue(expected.hasNext()); + assertEquals(expected.next(), t); + } + assertFalse(expected.hasNext()); + } + + public void testMergeEmpty() { + List> subs = Collections.emptyList(); + assertFalse(CoalescedDeletes.mergedIterator(subs).hasNext()); + + subs = new ArrayList>(); + subs.add(new PrefixCodedTerms.Builder().finish().iterator()); + subs.add(new PrefixCodedTerms.Builder().finish().iterator()); + Iterator merged = CoalescedDeletes.mergedIterator(subs); + assertFalse(merged.hasNext()); + } + + public void testMergeOne() { + Term t1 = new Term("foo", "a"); + PrefixCodedTerms.Builder b1 = new PrefixCodedTerms.Builder(); + b1.add(t1); + PrefixCodedTerms pb1 = b1.finish(); + + Term t2 = new Term("foo", "b"); + PrefixCodedTerms.Builder b2 = new PrefixCodedTerms.Builder(); + b2.add(t2); + PrefixCodedTerms pb2 = b2.finish(); + + List> subs = new ArrayList>(); + subs.add(pb1.iterator()); + subs.add(pb2.iterator()); + + Iterator merged = CoalescedDeletes.mergedIterator(subs); + assertTrue(merged.hasNext()); + assertEquals(t1, merged.next()); + assertTrue(merged.hasNext()); + assertEquals(t2, merged.next()); + } + + public void testMergeRandom() { + PrefixCodedTerms pb[] = new PrefixCodedTerms[_TestUtil.nextInt(random, 2, 10)]; + Set superSet = new TreeSet(); + + for (int i = 0; i < pb.length; i++) { + Set terms = new TreeSet(); + int nterms = _TestUtil.nextInt(random, 0, 10000); + for (int j = 0; j < nterms; j++) { + Term term = new Term(_TestUtil.randomUnicodeString(random, 2), _TestUtil.randomUnicodeString(random, 4)); + terms.add(term); + } + superSet.addAll(terms); + + PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder(); + for (Term ref: terms) { + b.add(ref); + } + pb[i] = b.finish(); + } + + List> subs = new ArrayList>(); + for (int i = 0; i < pb.length; i++) { + subs.add(pb[i].iterator()); + } + + Iterator expected = superSet.iterator(); + Iterator actual = CoalescedDeletes.mergedIterator(subs); + while (actual.hasNext()) { + assertTrue(expected.hasNext()); + assertEquals(expected.next(), actual.next()); + } + assertFalse(expected.hasNext()); + } +}