You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/09/26 04:22:18 UTC

svn commit: r1627701 - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/codecs/lucene40/ core/src/java/org/apache/lucene/codec...

Author: rmuir
Date: Fri Sep 26 02:22:18 2014
New Revision: 1627701

URL: http://svn.apache.org/r1627701
Log:
LUCENE-5969: take bitvector out back and shoot it

Added:
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
      - copied unchanged from r1627530, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java
      - copied unchanged from r1627593, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java   (with props)
Removed:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene40/
Modified:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java Fri Sep 26 02:22:18 2014
@@ -80,8 +80,7 @@ public final class CodecUtil {
    * @throws IOException If there is an I/O error writing to the underlying medium.
    * @throws IllegalArgumentException If the codec name is not simple ASCII, or is more than 127 characters in length
    */
-  public static void writeHeader(DataOutput out, String codec, int version)
-    throws IOException {
+  public static void writeHeader(DataOutput out, String codec, int version) throws IOException {
     BytesRef bytes = new BytesRef(codec);
     if (bytes.length != codec.length() || bytes.length >= 128) {
       throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
@@ -90,6 +89,38 @@ public final class CodecUtil {
     out.writeString(codec);
     out.writeInt(version);
   }
+  
+  /**
+   * Writes a codec header for a per-segment, which records both a string to
+   * identify the file, a version number, and the unique ID of the segment. 
+   * This header can be parsed and validated with 
+   * {@link #checkSegmentHeader(DataInput, String, int, int, String) checkSegmentHeader()}.
+   * <p>
+   * CodecSegmentHeader --&gt; CodecHeader,SegmentID
+   * <ul>
+   *    <li>CodecHeader --&gt; {@link #writeHeader}
+   *    <li>SegmentID   --&gt; {@link DataOutput#writeString String}.
+   *        Unique identifier for the segment.
+   * </ul>
+   * <p>
+   * Note that the length of a segment header depends only upon the
+   * name of the codec, so this length can be computed at any time
+   * with {@link #headerLength(String)}.
+   * 
+   * @param out Output stream
+   * @param codec String to identify this file. It should be simple ASCII, 
+   *              less than 128 characters in length.
+   * @param segmentID Unique identifier for the segment
+   * @param version Version number
+   * @throws IOException If there is an I/O error writing to the underlying medium.
+   * @throws IllegalArgumentException If the codec name is not simple ASCII, or is more than 127 characters in length
+   */
+  // nocommit: fix javadocs, add segmentLength()
+  public static void writeSegmentHeader(DataOutput out, String codec, int version, String segmentID) throws IOException {
+    writeHeader(out, codec, version);
+    // nocommit: improve encoding of this ID
+    out.writeString(segmentID);
+  }
 
   /**
    * Computes the length of a codec header.
@@ -129,9 +160,7 @@ public final class CodecUtil {
    * @throws IOException If there is an I/O error reading from the underlying medium.
    * @see #writeHeader(DataOutput, String, int)
    */
-  public static int checkHeader(DataInput in, String codec, int minVersion, int maxVersion)
-    throws IOException {
-
+  public static int checkHeader(DataInput in, String codec, int minVersion, int maxVersion) throws IOException {
     // Safety to guard against reading a bogus string:
     final int actualHeader = in.readInt();
     if (actualHeader != CODEC_MAGIC) {
@@ -162,6 +191,46 @@ public final class CodecUtil {
   }
   
   /**
+   * Reads and validates a header previously written with 
+   * {@link #writeSegmentHeader(DataOutput, String, int, String)}.
+   * <p>
+   * When reading a file, supply the expected <code>codec</code>,
+   * expected version range (<code>minVersion to maxVersion</code>),
+   * and segment ID.
+   * 
+   * @param in Input stream, positioned at the point where the
+   *        header was previously written. Typically this is located
+   *        at the beginning of the file.
+   * @param codec The expected codec name.
+   * @param minVersion The minimum supported expected version number.
+   * @param maxVersion The maximum supported expected version number.
+   * @param segmentID The expected segment this file belongs to.
+   * @return The actual version found, when a valid header is found 
+   *         that matches <code>codec</code>, with an actual version 
+   *         where <code>minVersion <= actual <= maxVersion</code>, 
+   *         and matching <code>segmentID</code>
+   *         Otherwise an exception is thrown.
+   * @throws CorruptIndexException If the first four bytes are not
+   *         {@link #CODEC_MAGIC}, or if the actual codec found is
+   *         not <code>codec</code>, or if the <code>segmentID</code>
+   *         does not match.
+   * @throws IndexFormatTooOldException If the actual version is less 
+   *         than <code>minVersion</code>.
+   * @throws IndexFormatTooNewException If the actual version is greater 
+   *         than <code>maxVersion</code>.
+   * @throws IOException If there is an I/O error reading from the underlying medium.
+   * @see #writeSegmentHeader(DataOutput, String, int, String)
+   */
+  public static int checkSegmentHeader(DataInput in, String codec, int minVersion, int maxVersion, String segmentID) throws IOException {
+    int version = checkHeader(in, codec, minVersion, maxVersion);
+    String id = in.readString();
+    if (!id.equals(segmentID)) {
+      throw new CorruptIndexException("file mismatch, expected segment id=" + segmentID + ", got=" + id, in);
+    }
+    return version;
+  }
+  
+  /**
    * Writes a codec footer, which records both a checksum
    * algorithm ID and a checksum. This footer can
    * be parsed and validated with 

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java Fri Sep 26 02:22:18 2014
@@ -27,7 +27,6 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
 import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
@@ -49,7 +48,7 @@ public class Lucene50Codec extends Codec
   private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
   private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
-  private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Fri Sep 26 02:22:18 2014
@@ -36,8 +36,7 @@ import org.apache.lucene.store.DataOutpu
  * FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup>,Footer</p>
  * <p>Data types:
  * <ul>
- *   <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
- *   <li>SegmentID --&gt; {@link DataOutput#writeString String}</li>
+ *   <li>Header --&gt; {@link CodecUtil#checkSegmentHeader SegmentHeader}</li>
  *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
  *   <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
@@ -49,7 +48,6 @@ import org.apache.lucene.store.DataOutpu
  * </p>
  * Field Descriptions:
  * <ul>
- *   <li>SegmentID: {@link SegmentInfo#getId()} this file belongs to</li>
  *   <li>FieldsCount: the number of fields in this file.</li>
  *   <li>FieldName: name of the field as a UTF-8 String.</li>
  *   <li>FieldNumber: the field's number. Note that unlike previous versions of

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosReader.java Fri Sep 26 02:22:18 2014
@@ -54,13 +54,10 @@ final class Lucene50FieldInfosReader ext
       Throwable priorE = null;
       FieldInfo infos[] = null;
       try {
-        CodecUtil.checkHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
+        CodecUtil.checkSegmentHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
                                      Lucene50FieldInfosFormat.FORMAT_START, 
-                                     Lucene50FieldInfosFormat.FORMAT_CURRENT);
-        String id = input.readString();
-        if (!id.equals(segmentInfo.getId())) {
-          throw new CorruptIndexException("file mismatch, expected segment id=" + segmentInfo.getId() + ", got=" + id, input);
-        }
+                                     Lucene50FieldInfosFormat.FORMAT_CURRENT,
+                                     segmentInfo.getId());
         
         final int size = input.readVInt(); //read in the size
         infos = new FieldInfo[size];

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosWriter.java Fri Sep 26 02:22:18 2014
@@ -47,8 +47,7 @@ final class Lucene50FieldInfosWriter ext
   public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene50FieldInfosFormat.EXTENSION);
     try (IndexOutput output = directory.createOutput(fileName, context)) {
-      CodecUtil.writeHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT);
-      output.writeString(segmentInfo.getId());
+      CodecUtil.writeSegmentHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId());
       output.writeVInt(infos.size());
       for (FieldInfo fi : infos) {
         fi.checkConsistency();

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java?rev=1627701&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java Fri Sep 26 02:22:18 2014
@@ -0,0 +1,115 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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.Collection;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.MutableBits;
+
+/** 
+ * Lucene 5.0 live docs format 
+ * <p>
+ * <p>The .liv file is optional, and only exists when a segment contains
+ * deletions.</p>
+ * <p>Although per-segment, this file is maintained exterior to compound segment
+ * files.</p>
+ * <p>Deletions (.liv) --&gt; SegmentHeader,Bits</p>
+ * <ul>
+ *   <li>SegmentHeader --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Bits --&gt; &lt;{@link DataOutput#writeLong Int64}&gt; <sup>LongCount</sup></li>
+ * </ul>
+ */
+public class Lucene50LiveDocsFormat extends LiveDocsFormat {
+  
+  /** extension of live docs */
+  private static final String EXTENSION = "liv";
+  
+  /** codec of live docs */
+  private static final String CODEC_NAME = "Lucene50LiveDocs";
+  
+  /** supported version range */
+  private static final int VERSION_START = 0;
+  private static final int VERSION_CURRENT = VERSION_START;
+
+  @Override
+  public MutableBits newLiveDocs(int size) throws IOException {
+    FixedBitSet bits = new FixedBitSet(size);
+    bits.set(0, size);
+    return bits;
+  }
+
+  @Override
+  public MutableBits newLiveDocs(Bits existing) throws IOException {
+    FixedBitSet fbs = (FixedBitSet) existing;
+    return fbs.clone();
+  }
+
+  @Override
+  public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
+    String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, info.getDelGen());
+    final int length = info.info.getDocCount();
+    try (ChecksumIndexInput input = dir.openChecksumInput(name, context)) {
+      Throwable priorE = null;
+      try {
+        CodecUtil.checkSegmentHeader(input, CODEC_NAME, VERSION_START, VERSION_CURRENT, info.info.getId());
+        long data[] = new long[FixedBitSet.bits2words(length)];
+        for (int i = 0; i < data.length; i++) {
+          data[i] = input.readLong();
+        }
+        return new FixedBitSet(data, length);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(input, priorE);
+      }
+    }
+    throw new AssertionError();
+  }
+
+  @Override
+  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
+    String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, info.getNextDelGen());
+    long data[] = ((FixedBitSet) bits).getBits();
+    try (IndexOutput output = dir.createOutput(name, context)) {
+      CodecUtil.writeSegmentHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId());
+      for (int i = 0; i < data.length; i++) {
+        output.writeLong(data[i]);
+      }
+      CodecUtil.writeFooter(output);
+    }
+  }
+
+  @Override
+  public void files(SegmentCommitInfo info, Collection<String> files) throws IOException {
+    if (info.hasDeletions()) {
+      files.add(IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, info.getDelGen()));
+    }
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java Fri Sep 26 02:22:18 2014
@@ -31,7 +31,7 @@ import org.apache.lucene.search.DocIdSet
  * 
  * @lucene.internal
  */
-public final class FixedBitSet extends DocIdSet implements Bits {
+public final class FixedBitSet extends DocIdSet implements MutableBits {
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);
 

Modified: lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1627701&r1=1627700&r2=1627701&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Fri Sep 26 02:22:18 2014
@@ -96,7 +96,7 @@ public class TestIndexFileDeleter extend
     */
 
     // TODO: fix this test better
-    String ext = Codec.getDefault().getName().equals("SimpleText") ? ".liv" : ".del";
+    String ext = ".liv";
     
     // Create a bogus separate del file for a
     // segment that already has a separate del file: