You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2021/07/09 16:50:08 UTC

[lucene] branch main updated: LUCENE-10019: Align file starts in CFS files to have proper alignment (8 bytes) (#203)

This is an automated email from the ASF dual-hosted git repository.

uschindler pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 69e8592  LUCENE-10019: Align file starts in CFS files to have proper alignment (8 bytes) (#203)
69e8592 is described below

commit 69e85924b7456b052bc87bcaf4793a6925f38dc5
Author: Uwe Schindler <us...@apache.org>
AuthorDate: Fri Jul 9 18:50:01 2021 +0200

    LUCENE-10019: Align file starts in CFS files to have proper alignment (8 bytes) (#203)
---
 .../codecs/lucene90/Lucene90CompoundFormat.java    |  6 +-
 .../codecs/lucene90/Lucene90CompoundReader.java    | 12 +--
 .../java/org/apache/lucene/store/IndexOutput.java  | 30 ++++++++
 .../lucene/index/TestSizeBoundedForceMerge.java    |  2 +
 .../lucene/store/TestIndexOutputAlignment.java     | 86 ++++++++++++++++++++++
 5 files changed, 129 insertions(+), 7 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundFormat.java
index f37b669..bca1721 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundFormat.java
@@ -60,7 +60,8 @@ import org.apache.lucene.store.IndexOutput;
  *   <li>FileCount indicates how many files are contained in this compound file. The entry table
  *       that follows has that many entries.
  *   <li>Each directory entry contains a long pointer to the start of this file's data section, the
- *       files length, and a String with that file's name.
+ *       files length, and a String with that file's name. The start of file's data section is
+ *       aligned to 8 bytes to not introduce additional unaligned accesses with mmap.
  * </ul>
  */
 public final class Lucene90CompoundFormat extends CompoundFormat {
@@ -106,8 +107,9 @@ public final class Lucene90CompoundFormat extends CompoundFormat {
     // write number of files
     entries.writeVInt(si.files().size());
     for (String file : si.files()) {
+      // align file start offset
+      long startOffset = data.alignFilePointer(Long.BYTES);
       // write bytes for file
-      long startOffset = data.getFilePointer();
       try (ChecksumIndexInput in = dir.openChecksumInput(file, IOContext.READONCE)) {
 
         // just copies the index header, verifying that its id matches what we expect
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
index cbf1e0d..9686375 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90CompoundReader.java
@@ -67,11 +67,13 @@ final class Lucene90CompoundReader extends CompoundDirectory {
     this.entries = readEntries(si.getId(), directory, entriesFileName);
     boolean success = false;
 
-    long expectedLength = CodecUtil.indexHeaderLength(Lucene90CompoundFormat.DATA_CODEC, "");
-    for (Map.Entry<String, FileEntry> ent : entries.entrySet()) {
-      expectedLength += ent.getValue().length;
-    }
-    expectedLength += CodecUtil.footerLength();
+    // find the last FileEntry in the map (largest offset+length) and add length of codec footer:
+    final long expectedLength =
+        entries.values().stream()
+                .mapToLong(e -> e.offset + e.length)
+                .max()
+                .orElseGet(() -> CodecUtil.indexHeaderLength(Lucene90CompoundFormat.DATA_CODEC, ""))
+            + CodecUtil.footerLength();
 
     handle = directory.openInput(dataFileName, context);
     try {
diff --git a/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java b/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
index 3674b38..4041ebd 100644
--- a/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
@@ -73,4 +73,34 @@ public abstract class IndexOutput extends DataOutput implements Closeable {
   public String toString() {
     return resourceDescription;
   }
+
+  /**
+   * Aligns the current file pointer to multiples of {@code alignmentBytes} bytes to improve reads
+   * with mmap. This will write between 0 and {@code (alignmentBytes-1)} zero bytes using {@link
+   * #writeByte(byte)}.
+   *
+   * @param alignmentBytes the alignment to which it should forward file pointer (must be a power of
+   *     2)
+   * @return the new file pointer after alignment
+   * @see #alignOffset(long, int)
+   */
+  public final long alignFilePointer(int alignmentBytes) throws IOException {
+    final long offset = getFilePointer(), alignedOffset = alignOffset(offset, alignmentBytes);
+    final int count = (int) (alignedOffset - offset);
+    for (int i = 0; i < count; i++) {
+      writeByte((byte) 0);
+    }
+    return alignedOffset;
+  }
+
+  /**
+   * Aligns the given {@code offset} to multiples of {@code alignmentBytes} bytes by rounding up.
+   * The alignment must be a power of 2.
+   */
+  public static final long alignOffset(long offset, int alignmentBytes) {
+    if (1 != Integer.bitCount(alignmentBytes)) {
+      throw new IllegalArgumentException("Alignment must be a power of 2");
+    }
+    return (offset - 1L + alignmentBytes) & (-alignmentBytes);
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java b/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java
index 99fb5a5..b4403a5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java
@@ -45,6 +45,8 @@ public class TestSizeBoundedForceMerge extends LuceneTestCase {
     IndexWriterConfig conf = newIndexWriterConfig(null);
     conf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH);
     conf.setRAMBufferSizeMB(IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
+    // don't use compound files, because the overhead make size checks unreliable.
+    conf.setUseCompoundFile(false);
     // prevent any merges by default.
     conf.setMergePolicy(NoMergePolicy.INSTANCE);
     return conf;
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestIndexOutputAlignment.java b/lucene/core/src/test/org/apache/lucene/store/TestIndexOutputAlignment.java
new file mode 100644
index 0000000..9ef7b33
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/store/TestIndexOutputAlignment.java
@@ -0,0 +1,86 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.store;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.stream.IntStream;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestIndexOutputAlignment extends LuceneTestCase {
+
+  public void testAlignmentCalculation() {
+    assertEquals(0L, IndexOutput.alignOffset(0L, Long.BYTES));
+    assertEquals(0L, IndexOutput.alignOffset(0L, Integer.BYTES));
+    assertEquals(0L, IndexOutput.alignOffset(0L, Short.BYTES));
+    assertEquals(0L, IndexOutput.alignOffset(0L, Byte.BYTES));
+
+    assertEquals(8L, IndexOutput.alignOffset(1L, Long.BYTES));
+    assertEquals(4L, IndexOutput.alignOffset(1L, Integer.BYTES));
+    assertEquals(2L, IndexOutput.alignOffset(1L, Short.BYTES));
+    assertEquals(1L, IndexOutput.alignOffset(1L, Byte.BYTES));
+
+    assertEquals(32L, IndexOutput.alignOffset(25L, Long.BYTES));
+    assertEquals(28L, IndexOutput.alignOffset(25L, Integer.BYTES));
+    assertEquals(26L, IndexOutput.alignOffset(25L, Short.BYTES));
+    assertEquals(25L, IndexOutput.alignOffset(25L, Byte.BYTES));
+
+    final long val = 1L << 48;
+    assertEquals(val, IndexOutput.alignOffset(val - 1, Long.BYTES));
+    assertEquals(val, IndexOutput.alignOffset(val - 1, Integer.BYTES));
+    assertEquals(val, IndexOutput.alignOffset(val - 1, Short.BYTES));
+    // byte alignment never changes anything:
+    assertEquals(val - 1, IndexOutput.alignOffset(val - 1, Byte.BYTES));
+  }
+
+  public void testInvalidAlignments() {
+    assertInvalidAligment(0);
+    assertInvalidAligment(-1);
+    assertInvalidAligment(-2);
+    assertInvalidAligment(6);
+    assertInvalidAligment(43);
+  }
+
+  private static void assertInvalidAligment(int size) {
+    assertThrows(IllegalArgumentException.class, () -> IndexOutput.alignOffset(1L, size));
+  }
+
+  public void testOutputAlignment() throws IOException {
+    IntStream.of(Long.BYTES, Integer.BYTES, Short.BYTES, Byte.BYTES)
+        .forEach(TestIndexOutputAlignment::runTestOutputAlignment);
+  }
+
+  private static void runTestOutputAlignment(int alignment) {
+    try (IndexOutput out =
+        new OutputStreamIndexOutput("test output", "test", new ByteArrayOutputStream(), 8192)) {
+      for (int i = 0; i < 10 * RANDOM_MULTIPLIER; i++) {
+        // write some bytes
+        int length = random().nextInt(32);
+        out.writeBytes(new byte[length], length);
+        long origPos = out.getFilePointer();
+        // align to next boundary
+        long newPos = out.alignFilePointer(alignment);
+        assertEquals(out.getFilePointer(), newPos);
+        assertTrue("not aligned", newPos % alignment == 0);
+        assertTrue("newPos >=", newPos >= origPos);
+        assertTrue("too much added", newPos - origPos < alignment);
+      }
+    } catch (IOException ioe) {
+      throw new AssertionError(ioe);
+    }
+  }
+}