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 649316ED6 for ; Mon, 30 May 2011 14:52:19 +0000 (UTC) Received: (qmail 72204 invoked by uid 500); 30 May 2011 14:52:19 -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 72197 invoked by uid 99); 30 May 2011 14:52:19 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 30 May 2011 14:52:19 +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; Mon, 30 May 2011 14:52:03 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id A0BC42388A3B; Mon, 30 May 2011 14:51:40 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1129205 [3/7] - in /lucene/dev/branches/solr2452: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/spellchecker/ dev-tools/idea/modules/suggest/ dev-tools/maven/lucene/contrib/ dev-tools/maven/lucene/contrib/spellc... Date: Mon, 30 May 2011 14:51:37 -0000 To: commits@lucene.apache.org From: sarowe@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110530145140.A0BC42388A3B@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentMerger.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon May 30 14:51:25 2011 @@ -27,13 +27,13 @@ import org.apache.lucene.document.Docume import org.apache.lucene.index.IndexReader.FieldOption; import org.apache.lucene.index.MergePolicy.MergeAbortedException; import org.apache.lucene.index.codecs.Codec; -import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.MergeState; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.MultiBits; import org.apache.lucene.util.ReaderUtil; @@ -46,10 +46,6 @@ import org.apache.lucene.util.ReaderUtil * @see #add */ final class SegmentMerger { - - /** norms header placeholder */ - static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1}; - private Directory directory; private String segment; private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL; @@ -124,6 +120,12 @@ final class SegmentMerger { return mergedDocs; } + /** + * NOTE: this method creates a compound file for all files returned by + * info.files(). While, generally, this may include separate norms and + * deletion files, this SegmentInfo must not reference such files when this + * method is called, because they are not allowed within a compound file. + */ final Collection createCompoundFile(String fileName, final SegmentInfo info) throws IOException { @@ -131,6 +133,10 @@ final class SegmentMerger { Collection files = info.files(); CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort); for (String file : files) { + assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) + : ".del file is not allowed in .cfs: " + file; + assert !IndexFileNames.isSeparateNormsFile(file) + : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file; cfsWriter.addFile(file); } @@ -139,7 +145,7 @@ final class SegmentMerger { return files; } - + private static void addIndexed(IndexReader reader, FieldInfos fInfos, Collection names, boolean storeTermVectors, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, @@ -540,14 +546,13 @@ final class SegmentMerger { } codec = segmentWriteState.segmentCodecs.codec(); final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState); - - // NOTE: this is silly, yet, necessary -- we create a - // MultiBits as our skip docs only to have it broken - // apart when we step through the docs enums in - // MultiDocsEnum. - mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts); - try { + // NOTE: this is silly, yet, necessary -- we create a + // MultiBits as our skip docs only to have it broken + // apart when we step through the docs enums in + // MultiDocsEnum. + mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts); + consumer.merge(mergeState, new MultiFields(fields.toArray(Fields.EMPTY_ARRAY), slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY))); @@ -573,12 +578,13 @@ final class SegmentMerger { private void mergeNorms() throws IOException { IndexOutput output = null; + boolean success = false; try { for (FieldInfo fi : fieldInfos) { if (fi.isIndexed && !fi.omitNorms) { if (output == null) { output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION)); - output.writeBytes(NORMS_HEADER,NORMS_HEADER.length); + output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length); } for (IndexReader reader : readers) { final int maxDoc = reader.maxDoc(); @@ -606,10 +612,9 @@ final class SegmentMerger { } } } + success = true; } finally { - if (output != null) { - output.close(); - } + IOUtils.closeSafely(!success, output); } } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentNorms.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentNorms.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentNorms.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentNorms.java Mon May 30 14:51:25 2011 @@ -33,6 +33,10 @@ import org.apache.lucene.store.IndexOutp */ final class SegmentNorms implements Cloneable { + + /** norms header placeholder */ + static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1}; + int refCount = 1; // If this instance is a clone, the originalNorm @@ -219,7 +223,7 @@ final class SegmentNorms implements Clon boolean success = false; try { try { - out.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length); + out.writeBytes(SegmentNorms.NORMS_HEADER, 0, SegmentNorms.NORMS_HEADER.length); out.writeBytes(bytes, owner.maxDoc()); } finally { out.close(); Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon May 30 14:51:25 2011 @@ -574,7 +574,7 @@ public class SegmentReader extends Index } private void openNorms(Directory cfsDir, int readBufferSize) throws IOException { - long nextNormSeek = SegmentMerger.NORMS_HEADER.length; //skip header (header unused for now) + long nextNormSeek = SegmentNorms.NORMS_HEADER.length; //skip header (header unused for now) int maxDoc = maxDoc(); for (FieldInfo fi : core.fieldInfos) { if (norms.containsKey(fi.name)) { @@ -619,7 +619,7 @@ public class SegmentReader extends Index if (isUnversioned) { normSeek = 0; } else { - normSeek = SegmentMerger.NORMS_HEADER.length; + normSeek = SegmentNorms.NORMS_HEADER.length; } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon May 30 14:51:25 2011 @@ -54,9 +54,7 @@ final class TermVectorsTermsWriter exten fill(state.numDocs); assert state.segmentName != null; String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION); - tvx.close(); - tvf.close(); - tvd.close(); + IOUtils.closeSafely(false, tvx, tvf, tvd); tvx = tvd = tvf = null; if (4+((long) state.numDocs)*16 != state.directory.fileLength(idxName)) { throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName)); @@ -89,18 +87,25 @@ final class TermVectorsTermsWriter exten private final void initTermVectorsWriter() throws IOException { if (tvx == null) { - - // If we hit an exception while init'ing the term - // vector output files, we must abort this segment - // because those files will be in an unknown - // state: - tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION)); - tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); - tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); - - tvx.writeInt(TermVectorsReader.FORMAT_CURRENT); - tvd.writeInt(TermVectorsReader.FORMAT_CURRENT); - tvf.writeInt(TermVectorsReader.FORMAT_CURRENT); + boolean success = false; + try { + // If we hit an exception while init'ing the term + // vector output files, we must abort this segment + // because those files will be in an unknown + // state: + tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION)); + tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); + tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); + + tvx.writeInt(TermVectorsReader.FORMAT_CURRENT); + tvd.writeInt(TermVectorsReader.FORMAT_CURRENT); + tvf.writeInt(TermVectorsReader.FORMAT_CURRENT); + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, tvx, tvd, tvf); + } + } lastDocID = 0; } @@ -139,7 +144,7 @@ final class TermVectorsTermsWriter exten } } - assert lastDocID == docState.docID; + assert lastDocID == docState.docID: "lastDocID=" + lastDocID + " docState.docID=" + docState.docID; lastDocID++; @@ -152,21 +157,27 @@ final class TermVectorsTermsWriter exten public void abort() { hasVectors = false; try { - IOUtils.closeSafely(tvx, tvd, tvf); - } catch (IOException ignored) { + IOUtils.closeSafely(true, tvx, tvd, tvf); + } catch (IOException e) { + // cannot happen since we suppress exceptions + throw new RuntimeException(e); } + try { docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION)); } catch (IOException ignored) { } + try { docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); } catch (IOException ignored) { } + try { docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); } catch (IOException ignored) { } + tvx = tvd = tvf = null; lastDocID = 0; Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java Mon May 30 14:51:25 2011 @@ -31,15 +31,22 @@ final class TermVectorsWriter { private FieldInfos fieldInfos; public TermVectorsWriter(Directory directory, String segment, - FieldInfos fieldInfos) - throws IOException { - // Open files for TermVector storage - tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION)); - tvx.writeInt(TermVectorsReader.FORMAT_CURRENT); - tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); - tvd.writeInt(TermVectorsReader.FORMAT_CURRENT); - tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); - tvf.writeInt(TermVectorsReader.FORMAT_CURRENT); + FieldInfos fieldInfos) throws IOException { + boolean success = false; + try { + // Open files for TermVector storage + tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION)); + tvx.writeInt(TermVectorsReader.FORMAT_CURRENT); + tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); + tvd.writeInt(TermVectorsReader.FORMAT_CURRENT); + tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); + tvf.writeInt(TermVectorsReader.FORMAT_CURRENT); + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, tvx, tvd, tvf); + } + } this.fieldInfos = fieldInfos; } @@ -51,8 +58,7 @@ final class TermVectorsWriter { * @param vectors * @throws IOException */ - public final void addAllDocVectors(TermFreqVector[] vectors) - throws IOException { + public final void addAllDocVectors(TermFreqVector[] vectors) throws IOException { tvx.writeLong(tvd.getFilePointer()); tvx.writeLong(tvf.getFilePointer()); @@ -187,6 +193,6 @@ final class TermVectorsWriter { final void close() throws IOException { // make an effort to close all streams we can but remember and re-throw // the first exception encountered in this process - IOUtils.closeSafely(tvx, tvd, tvf); + IOUtils.closeSafely(false, tvx, tvd, tvf); } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermsHash.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermsHash.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon May 30 14:51:25 2011 @@ -54,7 +54,6 @@ final class TermsHash extends InvertedDo final boolean trackAllocations; - public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) { this.docState = docWriter.docState; this.docWriter = docWriter; @@ -108,11 +107,11 @@ final class TermsHash extends InvertedDo } for (final Map.Entry entry : fieldsToFlush.entrySet()) { - TermsHashPerField perField = (TermsHashPerField) entry.getValue(); - childFields.put(entry.getKey(), perField.consumer); - if (nextTermsHash != null) { - nextChildFields.put(entry.getKey(), perField.nextPerField); - } + TermsHashPerField perField = (TermsHashPerField) entry.getValue(); + childFields.put(entry.getKey(), perField.consumer); + if (nextTermsHash != null) { + nextChildFields.put(entry.getKey(), perField.nextPerField); + } } consumer.flush(childFields, state); @@ -134,12 +133,9 @@ final class TermsHash extends InvertedDo @Override void finishDocument() throws IOException { - try { - consumer.finishDocument(this); - } finally { - if (nextTermsHash != null) { - nextTermsHash.consumer.finishDocument(nextTermsHash); - } + consumer.finishDocument(this); + if (nextTermsHash != null) { + nextTermsHash.consumer.finishDocument(nextTermsHash); } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java Mon May 30 14:51:25 2011 @@ -18,7 +18,6 @@ package org.apache.lucene.index; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.lucene.document.Document; import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; //javadoc /** @@ -48,12 +47,10 @@ public class ThreadAffinityDocumentsWrit } @Override - public ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc) { + public ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter) { ThreadState threadState = threadBindings.get(requestingThread); - if (threadState != null) { - if (threadState.tryLock()) { - return threadState; - } + if (threadState != null && threadState.tryLock()) { + return threadState; } ThreadState minThreadState = null; Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java Mon May 30 14:51:25 2011 @@ -31,6 +31,7 @@ import org.apache.lucene.store.RAMOutput import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; // TODO: currently we encode all terms between two indexed @@ -66,24 +67,29 @@ public class BlockTermsWriter extends Fi //private final String segment; - public BlockTermsWriter( - TermsIndexWriterBase termsIndexWriter, - SegmentWriteState state, - PostingsWriterBase postingsWriter) - throws IOException - { + public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter, + SegmentWriteState state, PostingsWriterBase postingsWriter) + throws IOException { final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION); this.termsIndexWriter = termsIndexWriter; out = state.directory.createOutput(termsFileName); - fieldInfos = state.fieldInfos; - writeHeader(out); - currentField = null; - this.postingsWriter = postingsWriter; - //segment = state.segmentName; - - //System.out.println("BTW.init seg=" + state.segmentName); - - postingsWriter.start(out); // have consumer write its format/header + boolean success = false; + try { + fieldInfos = state.fieldInfos; + writeHeader(out); + currentField = null; + this.postingsWriter = postingsWriter; + //segment = state.segmentName; + + //System.out.println("BTW.init seg=" + state.segmentName); + + postingsWriter.start(out); // have consumer write its format/header + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, out); + } + } } protected void writeHeader(IndexOutput out) throws IOException { @@ -130,20 +136,11 @@ public class BlockTermsWriter extends Fi } writeTrailer(dirStart); } finally { - try { - out.close(); - } finally { - try { - postingsWriter.close(); - } finally { - termsIndexWriter.close(); - } - } + IOUtils.closeSafely(false, out, postingsWriter, termsIndexWriter); } } protected void writeTrailer(long dirStart) throws IOException { - // TODO Auto-generated method stub out.seek(CodecUtil.headerLength(CODEC_NAME)); out.writeLong(dirStart); } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Mon May 30 14:51:25 2011 @@ -88,6 +88,15 @@ public class CodecProvider { return codec; } + /** + * Returns true iff a codec with the given name is registered + * @param name codec name + * @return true iff a codec with the given name is registered, otherwise false. + */ + public synchronized boolean isCodecRegistered(String name) { + return codecs.containsKey(name); + } + public SegmentInfosWriter getSegmentInfosWriter() { return infosWriter; } @@ -146,6 +155,14 @@ public class CodecProvider { } /** + * Returns true if this provider has a Codec registered for this + * field. + */ + public synchronized boolean hasFieldCodec(String name) { + return perFieldMap.containsKey(name); + } + + /** * Returns the default {@link Codec} for this {@link CodecProvider} * * @return the default {@link Codec} for this {@link CodecProvider} Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Mon May 30 14:51:25 2011 @@ -24,6 +24,7 @@ import org.apache.lucene.index.SegmentIn import org.apache.lucene.store.ChecksumIndexOutput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; /** * Default implementation of {@link SegmentInfosWriter}. @@ -56,16 +57,24 @@ public class DefaultSegmentInfosWriter e public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos) throws IOException { IndexOutput out = createOutput(dir, segmentFileName); - out.writeInt(FORMAT_CURRENT); // write FORMAT - out.writeLong(infos.version); - out.writeInt(infos.counter); // write counter - out.writeLong(infos.getGlobalFieldMapVersion()); - out.writeInt(infos.size()); // write infos - for (SegmentInfo si : infos) { - si.write(out); + boolean success = false; + try { + out.writeInt(FORMAT_CURRENT); // write FORMAT + out.writeLong(infos.version); + out.writeInt(infos.counter); // write counter + out.writeLong(infos.getGlobalFieldMapVersion()); + out.writeInt(infos.size()); // write infos + for (SegmentInfo si : infos) { + si.write(out); + } + out.writeStringStringMap(infos.getUserData()); + success = true; + return out; + } finally { + if (!success) { + IOUtils.closeSafely(true, out); + } } - out.writeStringStringMap(infos.getUserData()); - return out; } protected IndexOutput createOutput(Directory dir, String segmentFileName) Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Mon May 30 14:51:25 2011 @@ -24,6 +24,7 @@ import org.apache.lucene.index.FieldInfo import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.PagedBytes; import org.apache.lucene.util.packed.PackedInts; @@ -108,6 +109,7 @@ public class FixedGapTermsIndexReader ex } success = true; } finally { + if (!success) IOUtils.closeSafely(true, in); if (indexDivisor > 0) { in.close(); in = null; Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java Mon May 30 14:51:25 2011 @@ -25,6 +25,7 @@ import org.apache.lucene.index.SegmentWr import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.packed.PackedInts; import java.util.List; @@ -58,9 +59,17 @@ public class FixedGapTermsIndexWriter ex final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION); termIndexInterval = state.termIndexInterval; out = state.directory.createOutput(indexFileName); - fieldInfos = state.fieldInfos; - writeHeader(out); - out.writeInt(termIndexInterval); + boolean success = false; + try { + fieldInfos = state.fieldInfos; + writeHeader(out); + out.writeInt(termIndexInterval); + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, out); + } + } } protected void writeHeader(IndexOutput out) throws IOException { @@ -202,33 +211,37 @@ public class FixedGapTermsIndexWriter ex } } - @Override public void close() throws IOException { - final long dirStart = out.getFilePointer(); - final int fieldCount = fields.size(); - - int nonNullFieldCount = 0; - for(int i=0;i 0) { - nonNullFieldCount++; + boolean success = false; + try { + final long dirStart = out.getFilePointer(); + final int fieldCount = fields.size(); + + int nonNullFieldCount = 0; + for(int i=0;i 0) { + nonNullFieldCount++; + } } - } - - out.writeVInt(nonNullFieldCount); - for(int i=0;i 0) { - out.writeVInt(field.fieldInfo.number); - out.writeVInt(field.numIndexTerms); - out.writeVLong(field.termsStart); - out.writeVLong(field.indexStart); - out.writeVLong(field.packedIndexStart); - out.writeVLong(field.packedOffsetsStart); + + out.writeVInt(nonNullFieldCount); + for(int i=0;i 0) { + out.writeVInt(field.fieldInfo.number); + out.writeVInt(field.numIndexTerms); + out.writeVLong(field.termsStart); + out.writeVLong(field.indexStart); + out.writeVLong(field.packedIndexStart); + out.writeVLong(field.packedOffsetsStart); + } } + writeTrailer(dirStart); + success = true; + } finally { + IOUtils.closeSafely(!success, out); } - writeTrailer(dirStart); - out.close(); } protected void writeTrailer(long dirStart) throws IOException { Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java Mon May 30 14:51:25 2011 @@ -19,10 +19,12 @@ package org.apache.lucene.index.codecs; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.util.BytesRef; + +import java.io.Closeable; import java.io.IOException; /** @lucene.experimental */ -public abstract class TermsIndexWriterBase { +public abstract class TermsIndexWriterBase implements Closeable { public abstract class FieldWriter { public abstract boolean checkIndexTerm(BytesRef text, TermStats stats) throws IOException; @@ -31,6 +33,4 @@ public abstract class TermsIndexWriterBa } public abstract FieldWriter addField(FieldInfo fieldInfo, long termsFilePointer) throws IOException; - - public abstract void close() throws IOException; } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java Mon May 30 14:51:25 2011 @@ -33,11 +33,11 @@ import org.apache.lucene.store.Directory import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; -import org.apache.lucene.util.automaton.fst.Builder; -import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum; -import org.apache.lucene.util.automaton.fst.FST; -import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; -import org.apache.lucene.util.automaton.fst.Util; // for toDot +import org.apache.lucene.util.fst.Builder; +import org.apache.lucene.util.fst.BytesRefFSTEnum; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.PositiveIntOutputs; +import org.apache.lucene.util.fst.Util; // for toDot /** See {@link VariableGapTermsIndexWriter} * Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java Mon May 30 14:51:25 2011 @@ -28,9 +28,10 @@ import org.apache.lucene.index.SegmentWr import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; -import org.apache.lucene.util.automaton.fst.Builder; -import org.apache.lucene.util.automaton.fst.FST; -import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.fst.Builder; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.PositiveIntOutputs; /** * Selects index terms according to provided pluggable @@ -159,9 +160,17 @@ public class VariableGapTermsIndexWriter public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException { final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION); out = state.directory.createOutput(indexFileName); - fieldInfos = state.fieldInfos; - this.policy = policy; - writeHeader(out); + boolean success = false; + try { + fieldInfos = state.fieldInfos; + this.policy = policy; + writeHeader(out); + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, out); + } + } } protected void writeHeader(IndexOutput out) throws IOException { @@ -265,8 +274,8 @@ public class VariableGapTermsIndexWriter } } - @Override public void close() throws IOException { + try { final long dirStart = out.getFilePointer(); final int fieldCount = fields.size(); @@ -287,8 +296,10 @@ public class VariableGapTermsIndexWriter } } writeTrailer(dirStart); + } finally { out.close(); } + } protected void writeTrailer(long dirStart) throws IOException { out.seek(CodecUtil.headerLength(CODEC_NAME)); Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexOutput.java Mon May 30 14:51:25 2011 @@ -41,6 +41,7 @@ public abstract class VariableIntBlockIn protected final IndexOutput out; private int upto; + private boolean hitExcDuringWrite; // TODO what Var-Var codecs exist in practice... and what are there blocksizes like? // if its less than 128 we should set that as max and use byte? @@ -105,19 +106,23 @@ public abstract class VariableIntBlockIn @Override public void write(int v) throws IOException { + hitExcDuringWrite = true; upto -= add(v)-1; + hitExcDuringWrite = false; assert upto >= 0; } @Override public void close() throws IOException { try { - // stuff 0s in until the "real" data is flushed: - int stuffed = 0; - while(upto > stuffed) { - upto -= add(0)-1; - assert upto >= 0; - stuffed += 1; + if (!hitExcDuringWrite) { + // stuff 0s in until the "real" data is flushed: + int stuffed = 0; + while(upto > stuffed) { + upto -= add(0)-1; + assert upto >= 0; + stuffed += 1; + } } } finally { out.close(); Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Mon May 30 14:51:25 2011 @@ -38,6 +38,7 @@ import org.apache.lucene.index.codecs.Te import org.apache.lucene.index.codecs.TermsIndexWriterBase; import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; /** This codec "inlines" the postings for terms that have * low docFreq. It wraps another codec, which is used for @@ -81,7 +82,7 @@ public class PulsingCodec extends Codec success = true; } finally { if (!success) { - pulsingWriter.close(); + IOUtils.closeSafely(true, pulsingWriter); } } @@ -93,11 +94,7 @@ public class PulsingCodec extends Codec return ret; } finally { if (!success) { - try { - pulsingWriter.close(); - } finally { - indexWriter.close(); - } + IOUtils.closeSafely(true, pulsingWriter, indexWriter); } } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java Mon May 30 14:51:25 2011 @@ -71,8 +71,6 @@ public final class PulsingPostingsWriter * for this term) is <= maxPositions, then the postings are * inlined into terms dict */ public PulsingPostingsWriterImpl(int maxPositions, PostingsWriterBase wrappedPostingsWriter) throws IOException { - super(); - pending = new Position[maxPositions]; for(int i=0;i extensions) { Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Mon May 30 14:51:25 2011 @@ -29,13 +29,14 @@ import org.apache.lucene.index.FieldInfo import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.CharsRef; import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.UnicodeUtil; -import org.apache.lucene.util.automaton.fst.Builder; -import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum; -import org.apache.lucene.util.automaton.fst.FST; -import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; -import org.apache.lucene.util.automaton.fst.PairOutputs; +import org.apache.lucene.util.fst.Builder; +import org.apache.lucene.util.fst.BytesRefFSTEnum; +import org.apache.lucene.util.fst.FST; +import org.apache.lucene.util.fst.PositiveIntOutputs; +import org.apache.lucene.util.fst.PairOutputs; import java.io.IOException; import java.util.Comparator; @@ -236,7 +237,7 @@ class SimpleTextFieldsReader extends Fie private int tf; private Bits skipDocs; private final BytesRef scratch = new BytesRef(10); - private final UnicodeUtil.UTF16Result scratchUTF16 = new UnicodeUtil.UTF16Result(); + private final CharsRef scratchUTF16 = new CharsRef(10); public SimpleTextDocsEnum() { this.inStart = SimpleTextFieldsReader.this.in; @@ -286,7 +287,7 @@ class SimpleTextFieldsReader extends Fie return docID; } UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16); - docID = ArrayUtil.parseInt(scratchUTF16.result, 0, scratchUTF16.length); + docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length); termFreq = 0; first = false; } else if (scratch.startsWith(POS)) { @@ -323,8 +324,8 @@ class SimpleTextFieldsReader extends Fie private Bits skipDocs; private final BytesRef scratch = new BytesRef(10); private final BytesRef scratch2 = new BytesRef(10); - private final UnicodeUtil.UTF16Result scratchUTF16 = new UnicodeUtil.UTF16Result(); - private final UnicodeUtil.UTF16Result scratchUTF16_2 = new UnicodeUtil.UTF16Result(); + private final CharsRef scratchUTF16 = new CharsRef(10); + private final CharsRef scratchUTF16_2 = new CharsRef(10); private BytesRef payload; private long nextDocStart; @@ -368,7 +369,7 @@ class SimpleTextFieldsReader extends Fie return docID; } UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16); - docID = ArrayUtil.parseInt(scratchUTF16.result, 0, scratchUTF16.length); + docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length); tf = 0; posStart = in.getFilePointer(); first = false; @@ -400,7 +401,7 @@ class SimpleTextFieldsReader extends Fie readLine(in, scratch); assert scratch.startsWith(POS): "got line=" + scratch.utf8ToString(); UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+POS.length, scratch.length-POS.length, scratchUTF16_2); - final int pos = ArrayUtil.parseInt(scratchUTF16_2.result, 0, scratchUTF16_2.length); + final int pos = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length); final long fp = in.getFilePointer(); readLine(in, scratch); if (scratch.startsWith(PAYLOAD)) { Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Mon May 30 14:51:25 2011 @@ -143,8 +143,11 @@ class SimpleTextFieldsWriter extends Fie @Override public void close() throws IOException { - write(END); - newline(); - out.close(); + try { + write(END); + newline(); + } finally { + out.close(); + } } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Mon May 30 14:51:25 2011 @@ -33,6 +33,7 @@ import org.apache.lucene.store.IndexOutp import org.apache.lucene.store.RAMOutputStream; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; +import org.apache.lucene.util.IOUtils; /** @lucene.experimental */ public final class StandardPostingsWriter extends PostingsWriterBase { @@ -42,8 +43,8 @@ public final class StandardPostingsWrite final static int VERSION_START = 0; final static int VERSION_CURRENT = VERSION_START; - final IndexOutput freqOut; - final IndexOutput proxOut; + IndexOutput freqOut; + IndexOutput proxOut; final DefaultSkipListWriter skipListWriter; /** Expert: The fraction of TermDocs entries stored in skip tables, * used to accelerate {@link DocsEnum#advance(int)}. Larger values result in @@ -85,31 +86,35 @@ public final class StandardPostingsWrite public StandardPostingsWriter(SegmentWriteState state) throws IOException { this(state, DEFAULT_SKIP_INTERVAL); } + public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException { - super(); this.skipInterval = skipInterval; this.skipMinimum = skipInterval; /* set to the same for now */ //this.segment = state.segmentName; String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION); freqOut = state.directory.createOutput(fileName); - - if (state.fieldInfos.hasProx()) { - // At least one field does not omit TF, so create the - // prox file - fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION); - proxOut = state.directory.createOutput(fileName); - } else { - // Every field omits TF so we will write no prox file - proxOut = null; + boolean success = false; + try { + if (state.fieldInfos.hasProx()) { + // At least one field does not omit TF, so create the + // prox file + fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION); + proxOut = state.directory.createOutput(fileName); + } else { + // Every field omits TF so we will write no prox file + proxOut = null; + } + + totalNumDocs = state.numDocs; + + skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels, + state.numDocs, freqOut, proxOut); + success = true; + } finally { + if (!success) { + IOUtils.closeSafely(true, freqOut, proxOut); + } } - - totalNumDocs = state.numDocs; - - skipListWriter = new DefaultSkipListWriter(skipInterval, - maxSkipLevels, - state.numDocs, - freqOut, - proxOut); } @Override @@ -267,12 +272,6 @@ public final class StandardPostingsWrite @Override public void close() throws IOException { - try { - freqOut.close(); - } finally { - if (proxOut != null) { - proxOut.close(); - } - } + IOUtils.closeSafely(false, freqOut, proxOut); } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/search/CachingCollector.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/search/CachingCollector.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/search/CachingCollector.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/search/CachingCollector.java Mon May 30 14:51:25 2011 @@ -181,6 +181,7 @@ public abstract class CachingCollector e curUpto = 0; } cachedScorer.score = curScores[curUpto]; + cachedScorer.doc = curDocs[curUpto]; other.collect(curDocs[curUpto++]); } } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/FSDirectory.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/FSDirectory.java Mon May 30 14:51:25 2011 @@ -22,8 +22,6 @@ import java.io.FileNotFoundException; import java.io.FilenameFilter; import java.io.IOException; import java.io.RandomAccessFile; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; import java.util.Collection; import static java.util.Collections.synchronizedSet; @@ -111,15 +109,6 @@ import org.apache.lucene.util.Constants; * @see Directory */ public abstract class FSDirectory extends Directory { - private final static MessageDigest DIGESTER; - - static { - try { - DIGESTER = MessageDigest.getInstance("MD5"); - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e.toString(), e); - } - } /** * Default read chunk size. This is a conditional default: on 32bit JVMs, it defaults to 100 MB. On 64bit JVMs, it's @@ -337,12 +326,6 @@ public abstract class FSDirectory extend return openInput(name, BufferedIndexInput.BUFFER_SIZE); } - /** - * So we can do some byte-to-hexchar conversion below - */ - private static final char[] HEX_DIGITS = - {'0','1','2','3','4','5','6','7','8','9','a','b','c','d','e','f'}; - @Override public String getLockID() { ensureOpen(); @@ -353,19 +336,12 @@ public abstract class FSDirectory extend throw new RuntimeException(e.toString(), e); } - byte digest[]; - synchronized (DIGESTER) { - digest = DIGESTER.digest(dirName.getBytes()); - } - StringBuilder buf = new StringBuilder(); - buf.append("lucene-"); - for (int i = 0; i < digest.length; i++) { - int b = digest[i]; - buf.append(HEX_DIGITS[(b >> 4) & 0xf]); - buf.append(HEX_DIGITS[b & 0xf]); + int digest = 0; + for(int charIDX=0;charIDXclose() on */ public static void closeSafely(E priorException, Closeable... objects) throws E, IOException { - IOException firstIOE = null; + Throwable th = null; for (Closeable object : objects) { try { - if (object != null) + if (object != null) { object.close(); - } catch (IOException ioe) { - if (firstIOE == null) - firstIOE = ioe; + } + } catch (Throwable t) { + if (th == null) { + th = t; + } } } - if (priorException != null) + if (priorException != null) { throw priorException; - else if (firstIOE != null) - throw firstIOE; + } else if (th != null) { + if (th instanceof IOException) throw (IOException) th; + if (th instanceof RuntimeException) throw (RuntimeException) th; + if (th instanceof Error) throw (Error) th; + throw new RuntimeException(th); + } + } + + /** @see #closeSafely(Exception, Closeable...) */ + public static void closeSafely(E priorException, Iterable objects) throws E, IOException { + Throwable th = null; + + for (Closeable object : objects) { + try { + if (object != null) { + object.close(); + } + } catch (Throwable t) { + if (th == null) { + th = t; + } + } + } + + if (priorException != null) { + throw priorException; + } else if (th != null) { + if (th instanceof IOException) throw (IOException) th; + if (th instanceof RuntimeException) throw (RuntimeException) th; + if (th instanceof Error) throw (Error) th; + throw new RuntimeException(th); + } } /** - *

Closes all given Closeables, suppressing all thrown exceptions. Some of the Closeables - * may be null, they are ignored. After everything is closed, method either throws the first of suppressed exceptions, - * or completes normally.

- * @param objects objects to call close() on + * Closes all given Closeables, suppressing all thrown exceptions. + * Some of the Closeables may be null, they are ignored. After + * everything is closed, and if {@code suppressExceptions} is {@code false}, + * method either throws the first of suppressed exceptions, or completes + * normally. + * + * @param suppressExceptions + * if true then exceptions that occur during close() are suppressed + * @param objects + * objects to call close() on */ - public static void closeSafely(Closeable... objects) throws IOException { - IOException firstIOE = null; + public static void closeSafely(boolean suppressExceptions, Closeable... objects) throws IOException { + Throwable th = null; for (Closeable object : objects) { try { - if (object != null) + if (object != null) { object.close(); - } catch (IOException ioe) { - if (firstIOE == null) - firstIOE = ioe; + } + } catch (Throwable t) { + if (th == null) + th = t; } } - if (firstIOE != null) - throw firstIOE; + if (th != null && !suppressExceptions) { + if (th instanceof IOException) throw (IOException) th; + if (th instanceof RuntimeException) throw (RuntimeException) th; + if (th instanceof Error) throw (Error) th; + throw new RuntimeException(th); + } } + + /** + * @see #closeSafely(boolean, Closeable...) + */ + public static void closeSafely(boolean suppressExceptions, Iterable objects) throws IOException { + Throwable th = null; + + for (Closeable object : objects) { + try { + if (object != null) { + object.close(); + } + } catch (Throwable t) { + if (th == null) + th = t; + } + } + + if (th != null && !suppressExceptions) { + if (th instanceof IOException) throw (IOException) th; + if (th instanceof RuntimeException) throw (RuntimeException) th; + if (th instanceof Error) throw (Error) th; + throw new RuntimeException(th); + } + } + } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/OpenBitSet.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/OpenBitSet.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/OpenBitSet.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/OpenBitSet.java Mon May 30 14:51:25 2011 @@ -78,11 +78,15 @@ public class OpenBitSet extends DocIdSet protected long[] bits; protected int wlen; // number of words (elements) used in the array + // Used only for assert: + private long numBits; + /** Constructs an OpenBitSet large enough to hold numBits. * * @param numBits */ public OpenBitSet(long numBits) { + this.numBits = numBits; bits = new long[bits2words(numBits)]; wlen = bits.length; } @@ -107,6 +111,7 @@ public class OpenBitSet extends DocIdSet public OpenBitSet(long[] bits, int numWords) { this.bits = bits; this.wlen = numWords; + this.numBits = wlen * 64; } @Override @@ -170,6 +175,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size */ public boolean fastGet(int index) { + assert index >= 0 && index < numBits; int i = index >> 6; // div 64 // signed shift will keep a negative index and force an // array-index-out-of-bounds-exception, removing the need for an explicit check. @@ -194,6 +200,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public boolean fastGet(long index) { + assert index >= 0 && index < numBits; int i = (int)(index >> 6); // div 64 int bit = (int)index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -217,6 +224,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size */ public int getBit(int index) { + assert index >= 0 && index < numBits; int i = index >> 6; // div 64 int bit = index & 0x3f; // mod 64 return ((int)(bits[i]>>>bit)) & 0x01; @@ -245,6 +253,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastSet(int index) { + assert index >= 0 && index < numBits; int wordNum = index >> 6; // div 64 int bit = index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -255,6 +264,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastSet(long index) { + assert index >= 0 && index < numBits; int wordNum = (int)(index >> 6); int bit = (int)index & 0x3f; long bitmask = 1L << bit; @@ -296,6 +306,7 @@ public class OpenBitSet extends DocIdSet ensureCapacity(index+1); wlen = wordNum+1; } + assert (numBits = Math.max(numBits, index+1)) >= 0; return wordNum; } @@ -304,6 +315,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastClear(int index) { + assert index >= 0 && index < numBits; int wordNum = index >> 6; int bit = index & 0x03f; long bitmask = 1L << bit; @@ -321,6 +333,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastClear(long index) { + assert index >= 0 && index < numBits; int wordNum = (int)(index >> 6); // div 64 int bit = (int)index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -415,6 +428,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public boolean getAndSet(int index) { + assert index >= 0 && index < numBits; int wordNum = index >> 6; // div 64 int bit = index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -427,6 +441,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public boolean getAndSet(long index) { + assert index >= 0 && index < numBits; int wordNum = (int)(index >> 6); // div 64 int bit = (int)index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -439,6 +454,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastFlip(int index) { + assert index >= 0 && index < numBits; int wordNum = index >> 6; // div 64 int bit = index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -449,6 +465,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public void fastFlip(long index) { + assert index >= 0 && index < numBits; int wordNum = (int)(index >> 6); // div 64 int bit = (int)index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -467,6 +484,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public boolean flipAndGet(int index) { + assert index >= 0 && index < numBits; int wordNum = index >> 6; // div 64 int bit = index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -478,6 +496,7 @@ public class OpenBitSet extends DocIdSet * The index should be less than the OpenBitSet size. */ public boolean flipAndGet(long index) { + assert index >= 0 && index < numBits; int wordNum = (int)(index >> 6); // div 64 int bit = (int)index & 0x3f; // mod 64 long bitmask = 1L << bit; @@ -674,6 +693,7 @@ public class OpenBitSet extends DocIdSet public void union(OpenBitSet other) { int newLen = Math.max(wlen,other.wlen); ensureCapacityWords(newLen); + assert (numBits = Math.max(other.numBits, numBits)) >= 0; long[] thisArr = this.bits; long[] otherArr = other.bits; @@ -702,6 +722,7 @@ public class OpenBitSet extends DocIdSet public void xor(OpenBitSet other) { int newLen = Math.max(wlen,other.wlen); ensureCapacityWords(newLen); + assert (numBits = Math.max(other.numBits, numBits)) >= 0; long[] thisArr = this.bits; long[] otherArr = other.bits; Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/UnicodeUtil.java Mon May 30 14:51:25 2011 @@ -94,6 +94,19 @@ package org.apache.lucene.util; */ public final class UnicodeUtil { + + /** A binary term consisting of a number of 0xff bytes, likely to be bigger than other terms + * one would normally encounter, and definitely bigger than any UTF-8 terms. + *

+ * WARNING: This is not a valid UTF8 Term + **/ + public static final BytesRef BIG_TERM = new BytesRef( + new byte[] {-1,-1,-1,-1,-1,-1,-1,-1,-1,-1} + ); // TODO this is unrelated here find a better place for it + + public static void main(String[] args) { + System.out.println(Character.toChars(0x10FFFF + 1)); + } private UnicodeUtil() {} // no instance @@ -112,33 +125,6 @@ public final class UnicodeUtil { Character.MIN_SUPPLEMENTARY_CODE_POINT - (UNI_SUR_HIGH_START << HALF_SHIFT) - UNI_SUR_LOW_START; - /** - * @lucene.internal - */ - public static final class UTF16Result { - public char[] result = new char[10]; - public int[] offsets = new int[10]; - public int length; - - public void setLength(int newLength) { - if (result.length < newLength) - result = ArrayUtil.grow(result, newLength); - length = newLength; - } - - public void copyText(UTF16Result other) { - setLength(other.length); - System.arraycopy(other.result, 0, result, 0, length); - } - - public void copyText(String other) { - final int otherLength = other.length(); - setLength(otherLength); - other.getChars(0, otherLength, result, 0); - length = otherLength; - } - } - /** Encode characters from a char[] source, starting at * offset for length chars. Returns a hash of the resulting bytes. After encoding, result.offset will always be 0. */ public static int UTF16toUTF8WithHash(final char[] source, final int offset, final int length, BytesRef result) { @@ -302,135 +288,6 @@ public final class UnicodeUtil { result.length = upto; } - /** Convert UTF8 bytes into UTF16 characters. If offset - * is non-zero, conversion starts at that starting point - * in utf8, re-using the results from the previous call - * up until offset. */ - public static void UTF8toUTF16(final byte[] utf8, final int offset, final int length, final UTF16Result result) { - - final int end = offset + length; - char[] out = result.result; - if (result.offsets.length <= end) { - result.offsets = ArrayUtil.grow(result.offsets, end+1); - } - final int[] offsets = result.offsets; - - // If incremental decoding fell in the middle of a - // single unicode character, rollback to its start: - int upto = offset; - while(offsets[upto] == -1) - upto--; - - int outUpto = offsets[upto]; - - // Pre-allocate for worst case 1-for-1 - if (outUpto+length >= out.length) { - out = result.result = ArrayUtil.grow(out, outUpto+length+1); - } - - while (upto < end) { - - final int b = utf8[upto]&0xff; - final int ch; - - offsets[upto++] = outUpto; - - if (b < 0xc0) { - assert b < 0x80; - ch = b; - } else if (b < 0xe0) { - ch = ((b&0x1f)<<6) + (utf8[upto]&0x3f); - offsets[upto++] = -1; - } else if (b < 0xf0) { - ch = ((b&0xf)<<12) + ((utf8[upto]&0x3f)<<6) + (utf8[upto+1]&0x3f); - offsets[upto++] = -1; - offsets[upto++] = -1; - } else { - assert b < 0xf8; - ch = ((b&0x7)<<18) + ((utf8[upto]&0x3f)<<12) + ((utf8[upto+1]&0x3f)<<6) + (utf8[upto+2]&0x3f); - offsets[upto++] = -1; - offsets[upto++] = -1; - offsets[upto++] = -1; - } - - if (ch <= UNI_MAX_BMP) { - // target is a character <= 0xFFFF - out[outUpto++] = (char) ch; - } else { - // target is a character in range 0xFFFF - 0x10FFFF - out[outUpto++] = (char) ((ch >> HALF_SHIFT) + 0xD7C0 /* UNI_SUR_HIGH_START - 64 */); - out[outUpto++] = (char) ((ch & HALF_MASK) + UNI_SUR_LOW_START); - } - } - offsets[upto] = outUpto; - result.length = outUpto; - } - - /** - * Get the next valid UTF-16 String in UTF-16 order. - *

- * If the input String is already valid, it is returned. - * Otherwise the next String in code unit order is returned. - *

- * @param s input String (possibly with unpaired surrogates) - * @return next valid UTF-16 String in UTF-16 order - */ - public static String nextValidUTF16String(String s) { - if (validUTF16String(s)) - return s; - else { - UTF16Result chars = new UTF16Result(); - chars.copyText(s); - nextValidUTF16String(chars); - return new String(chars.result, 0, chars.length); - } - } - - public static void nextValidUTF16String(UTF16Result s) { - final int size = s.length; - for (int i = 0; i < size; i++) { - char ch = s.result[i]; - if (ch >= UnicodeUtil.UNI_SUR_HIGH_START - && ch <= UnicodeUtil.UNI_SUR_HIGH_END) { - if (i < size - 1) { - i++; - char nextCH = s.result[i]; - if (nextCH >= UnicodeUtil.UNI_SUR_LOW_START - && nextCH <= UnicodeUtil.UNI_SUR_LOW_END) { - // Valid surrogate pair - } else - // Unmatched high surrogate - if (nextCH < UnicodeUtil.UNI_SUR_LOW_START) { // SMP not enumerated - s.setLength(i + 1); - s.result[i] = (char) UnicodeUtil.UNI_SUR_LOW_START; - return; - } else { // SMP already enumerated - if (s.result[i - 1] == UnicodeUtil.UNI_SUR_HIGH_END) { - s.result[i - 1] = (char) (UnicodeUtil.UNI_SUR_LOW_END + 1); - s.setLength(i); - } else { - s.result[i - 1]++; - s.result[i] = (char) UnicodeUtil.UNI_SUR_LOW_START; - s.setLength(i + 1); - } - return; - } - } else { - // Unmatched high surrogate in final position, SMP not yet enumerated - s.setLength(i + 2); - s.result[i + 1] = (char) UnicodeUtil.UNI_SUR_LOW_START; - return; - } - } else if (ch >= UnicodeUtil.UNI_SUR_LOW_START - && ch <= UnicodeUtil.UNI_SUR_LOW_END) { - // Unmatched low surrogate, SMP already enumerated - s.setLength(i + 1); - s.result[i] = (char) (UnicodeUtil.UNI_SUR_LOW_END + 1); - return; - } - } - } - // Only called from assert /* private static boolean matches(char[] source, int offset, int length, byte[] result, int upto) { @@ -705,4 +562,51 @@ public final class UnicodeUtil { } return sb.toString(); } + + /** + * Interprets the given byte array as UTF-8 and converts to UTF-16. The {@link CharsRef} will be extended if + * it doesn't provide enough space to hold the worst case of each byte becoming a UTF-16 codepoint. + *

+ * NOTE: Full characters are read, even if this reads past the length passed (and + * can result in an ArrayOutOfBoundsException if invalid UTF-8 is passed). + * Explicit checks for valid UTF-8 are not performed. + */ + public static void UTF8toUTF16(byte[] utf8, int offset, int length, CharsRef chars) { + int out_offset = chars.offset = 0; + final char[] out = chars.chars = ArrayUtil.grow(chars.chars, length); + final int limit = offset + length; + while (offset < limit) { + int b = utf8[offset++]&0xff; + if (b < 0xc0) { + assert b < 0x80; + out[out_offset++] = (char)b; + } else if (b < 0xe0) { + out[out_offset++] = (char)(((b&0x1f)<<6) + (utf8[offset++]&0x3f)); + } else if (b < 0xf0) { + out[out_offset++] = (char)(((b&0xf)<<12) + ((utf8[offset]&0x3f)<<6) + (utf8[offset+1]&0x3f)); + offset += 2; + } else { + assert b < 0xf8; + int ch = ((b&0x7)<<18) + ((utf8[offset]&0x3f)<<12) + ((utf8[offset+1]&0x3f)<<6) + (utf8[offset+2]&0x3f); + offset += 3; + if (ch < UNI_MAX_BMP) { + out[out_offset++] = (char)ch; + } else { + int chHalf = ch - 0x0010000; + out[out_offset++] = (char) ((chHalf >> 10) + 0xD800); + out[out_offset++] = (char) ((chHalf & HALF_MASK) + 0xDC00); + } + } + } + chars.length = out_offset - chars.offset; + } + + /** + * Utility method for {@link #UTF8toUTF16(byte[], int, int, CharsRef)} + * @see #UTF8toUTF16(byte[], int, int, CharsRef) + */ + public static void UTF8toUTF16(BytesRef bytesRef, CharsRef chars) { + UTF8toUTF16(bytesRef.bytes, bytesRef.offset, bytesRef.length, chars); + } + } Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/Version.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/Version.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/Version.java (original) +++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/Version.java Mon May 30 14:51:25 2011 @@ -48,6 +48,13 @@ public enum Version { */ @Deprecated LUCENE_32, + + /** + * Match settings and bugs in Lucene's 3.3 release. + * @deprecated (4.0) Use latest + */ + @Deprecated + LUCENE_33, /** Match settings and bugs in Lucene's 4.0 release. *

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original) +++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon May 30 14:51:25 2011 @@ -19,6 +19,7 @@ package org.apache.lucene.index; import java.io.Closeable; import java.io.IOException; +import java.util.Iterator; import java.util.Random; import org.apache.lucene.analysis.Analyzer; @@ -97,8 +98,43 @@ public class RandomIndexWriter implement * Adds a Document. * @see IndexWriter#addDocument(Document) */ - public void addDocument(Document doc) throws IOException { - w.addDocument(doc); + public void addDocument(final Document doc) throws IOException { + if (r.nextInt(5) == 3) { + // TODO: maybe, we should simply buffer up added docs + // (but we need to clone them), and only when + // getReader, commit, etc. are called, we do an + // addDocuments? Would be better testing. + w.addDocuments(new Iterable() { + + // @Override -- not until Java 1.6 + public Iterator iterator() { + return new Iterator() { + boolean done; + + // @Override -- not until Java 1.6 + public boolean hasNext() { + return !done; + } + + // @Override -- not until Java 1.6 + public void remove() { + throw new UnsupportedOperationException(); + } + + // @Override -- not until Java 1.6 + public Document next() { + if (done) { + throw new IllegalStateException(); + } + done = true; + return doc; + } + }; + } + }); + } else { + w.addDocument(doc); + } maybeCommit(); } @@ -116,12 +152,53 @@ public class RandomIndexWriter implement } } + public void addDocuments(Iterable docs) throws IOException { + w.addDocuments(docs); + maybeCommit(); + } + + public void updateDocuments(Term delTerm, Iterable docs) throws IOException { + w.updateDocuments(delTerm, docs); + maybeCommit(); + } + /** * Updates a document. * @see IndexWriter#updateDocument(Term, Document) */ - public void updateDocument(Term t, Document doc) throws IOException { - w.updateDocument(t, doc); + public void updateDocument(Term t, final Document doc) throws IOException { + if (r.nextInt(5) == 3) { + w.updateDocuments(t, new Iterable() { + + // @Override -- not until Java 1.6 + public Iterator iterator() { + return new Iterator() { + boolean done; + + // @Override -- not until Java 1.6 + public boolean hasNext() { + return !done; + } + + // @Override -- not until Java 1.6 + public void remove() { + throw new UnsupportedOperationException(); + } + + // @Override -- not until Java 1.6 + public Document next() { + if (done) { + throw new IllegalStateException(); + } + done = true; + return doc; + } + }; + } + }); + } else { + w.updateDocument(t, doc); + } maybeCommit(); } Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1129205&r1=1129204&r2=1129205&view=diff ============================================================================== --- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original) +++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Mon May 30 14:51:25 2011 @@ -44,6 +44,7 @@ import org.apache.lucene.index.codecs.Te import org.apache.lucene.index.codecs.standard.StandardCodec; import org.apache.lucene.store.*; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOUtils; /** * A silly test codec to verify core support for fixed @@ -97,15 +98,25 @@ public class MockFixedIntBlockCodec exte @Override public IntIndexOutput createOutput(Directory dir, String fileName) throws IOException { - return new FixedIntBlockIndexOutput(dir.createOutput(fileName), blockSize) { - @Override - protected void flushBlock() throws IOException { - for(int i=0;i= 0; - out.writeVInt(buffer[i]); + IndexOutput out = dir.createOutput(fileName); + boolean success = false; + try { + FixedIntBlockIndexOutput ret = new FixedIntBlockIndexOutput(out, blockSize) { + @Override + protected void flushBlock() throws IOException { + for(int i=0;i= 0; + out.writeVInt(buffer[i]); + } } + }; + success = true; + return ret; + } finally { + if (!success) { + IOUtils.closeSafely(true, out); } - }; + } } }