You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2021/01/06 21:17:14 UTC

[lucene-solr] branch master updated: LUCENE-9652: DataInput.readLEFloats for use by Lucene90VectorReader (#2175)

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

sokolov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 7b9f875  LUCENE-9652: DataInput.readLEFloats for use by Lucene90VectorReader (#2175)
7b9f875 is described below

commit 7b9f87514561258029c9d97e7fc432bf73373f06
Author: Michael Sokolov <so...@falutin.net>
AuthorDate: Wed Jan 6 16:16:56 2021 -0500

    LUCENE-9652: DataInput.readLEFloats for use by Lucene90VectorReader (#2175)
---
 .../codecs/lucene90/Lucene90VectorReader.java      | 77 ++++++----------------
 .../codecs/lucene90/Lucene90VectorWriter.java      | 10 ++-
 .../apache/lucene/index/VectorValuesWriter.java    |  5 +-
 .../org/apache/lucene/store/ByteBufferGuard.java   |  6 ++
 .../apache/lucene/store/ByteBufferIndexInput.java  | 34 ++++++++++
 .../apache/lucene/store/ByteBuffersDataInput.java  | 51 +++++++++++++-
 .../apache/lucene/store/ByteBuffersIndexInput.java |  6 ++
 .../java/org/apache/lucene/store/DataInput.java    | 14 ++++
 .../org/apache/lucene/store/TestMultiMMap.java     | 19 ++++++
 .../apache/lucene/store/BaseDirectoryTestCase.java | 58 ++++++++++++++++
 .../apache/lucene/store/MockIndexInputWrapper.java |  6 ++
 11 files changed, 225 insertions(+), 61 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
index efb50a3..cab9922 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorReader.java
@@ -316,8 +316,8 @@ public final class Lucene90VectorReader extends VectorReader {
   }
 
   /** Read the vector values from the index input. This supports both iterated and random access. */
-  private final class OffHeapVectorValues extends VectorValues
-      implements RandomAccessVectorValuesProducer {
+  private class OffHeapVectorValues extends VectorValues
+      implements RandomAccessVectorValues, RandomAccessVectorValuesProducer {
 
     final FieldEntry fieldEntry;
     final IndexInput dataIn;
@@ -358,16 +358,15 @@ public final class Lucene90VectorReader extends VectorReader {
 
     @Override
     public float[] vectorValue() throws IOException {
-      binaryValue();
-      floatBuffer.position(0);
-      floatBuffer.get(value, 0, fieldEntry.dimension);
+      dataIn.seek(ord * byteSize);
+      dataIn.readLEFloats(value, 0, value.length);
       return value;
     }
 
     @Override
     public BytesRef binaryValue() throws IOException {
       dataIn.seek(ord * byteSize);
-      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize, false);
       return binaryValue;
     }
 
@@ -399,7 +398,7 @@ public final class Lucene90VectorReader extends VectorReader {
 
     @Override
     public RandomAccessVectorValues randomAccess() {
-      return new OffHeapRandomAccess(dataIn.clone());
+      return new OffHeapVectorValues(fieldEntry, dataIn.clone());
     }
 
     @Override
@@ -428,57 +427,23 @@ public final class Lucene90VectorReader extends VectorReader {
           scoreDocs);
     }
 
-    class OffHeapRandomAccess implements RandomAccessVectorValues {
-
-      final IndexInput dataIn;
-
-      final BytesRef binaryValue;
-      final ByteBuffer byteBuffer;
-      final FloatBuffer floatBuffer;
-      final float[] value;
-
-      OffHeapRandomAccess(IndexInput dataIn) {
-        this.dataIn = dataIn;
-        byteBuffer = ByteBuffer.allocate(byteSize);
-        floatBuffer = byteBuffer.asFloatBuffer();
-        value = new float[dimension()];
-        binaryValue = new BytesRef(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
-      }
-
-      @Override
-      public int size() {
-        return fieldEntry.size();
-      }
-
-      @Override
-      public int dimension() {
-        return fieldEntry.dimension;
-      }
-
-      @Override
-      public SearchStrategy searchStrategy() {
-        return fieldEntry.searchStrategy;
-      }
-
-      @Override
-      public float[] vectorValue(int targetOrd) throws IOException {
-        readValue(targetOrd);
-        floatBuffer.position(0);
-        floatBuffer.get(value);
-        return value;
-      }
+    @Override
+    public float[] vectorValue(int targetOrd) throws IOException {
+      dataIn.seek(targetOrd * byteSize);
+      dataIn.readLEFloats(value, 0, value.length);
+      return value;
+    }
 
-      @Override
-      public BytesRef binaryValue(int targetOrd) throws IOException {
-        readValue(targetOrd);
-        return binaryValue;
-      }
+    @Override
+    public BytesRef binaryValue(int targetOrd) throws IOException {
+      readValue(targetOrd);
+      return binaryValue;
+    }
 
-      private void readValue(int targetOrd) throws IOException {
-        long offset = targetOrd * byteSize;
-        dataIn.seek(offset);
-        dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
-      }
+    private void readValue(int targetOrd) throws IOException {
+      long offset = targetOrd * byteSize;
+      dataIn.seek(offset);
+      dataIn.readBytes(byteBuffer.array(), byteBuffer.arrayOffset(), byteSize);
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorWriter.java
index ec9c0c9..84be279 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90VectorWriter.java
@@ -96,8 +96,14 @@ public final class Lucene90VectorWriter extends VectorWriter {
 
   @Override
   public void writeField(FieldInfo fieldInfo, VectorValues vectors) throws IOException {
-    long vectorDataOffset = vectorData.getFilePointer();
-
+    long pos = vectorData.getFilePointer();
+    // write floats aligned at 4 bytes. This will not survive CFS, but it shows a small benefit when
+    // CFS is not used, eg for larger indexes
+    long padding = (4 - (pos & 0x3)) & 0x3;
+    long vectorDataOffset = pos + padding;
+    for (int i = 0; i < padding; i++) {
+      vectorData.writeByte((byte) 0);
+    }
     // TODO - use a better data structure; a bitset? DocsWithFieldSet is p.p. in o.a.l.index
     int[] docIds = new int[vectors.size()];
     int count = 0;
diff --git a/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
index a7f5f6b..e4c74a7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/VectorValuesWriter.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.lucene.codecs.VectorWriter;
@@ -269,9 +270,9 @@ class VectorValuesWriter {
       this.vectors = vectors;
       this.dimension = dimension;
       this.searchStrategy = searchStrategy;
-      buffer = ByteBuffer.allocate(dimension * Float.BYTES);
+      buffer = ByteBuffer.allocate(dimension * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
       binaryValue = new BytesRef(buffer.array());
-      raBuffer = ByteBuffer.allocate(dimension * Float.BYTES);
+      raBuffer = ByteBuffer.allocate(dimension * Float.BYTES).order(ByteOrder.LITTLE_ENDIAN);
       raBinaryValue = new BytesRef(raBuffer.array());
       docsWithFieldIter = docsWithField.iterator();
     }
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBufferGuard.java b/lucene/core/src/java/org/apache/lucene/store/ByteBufferGuard.java
index e880f02..68658d6 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBufferGuard.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBufferGuard.java
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.FloatBuffer;
 import java.nio.LongBuffer;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -136,4 +137,9 @@ final class ByteBufferGuard {
     ensureValid();
     receiver.get(dst, offset, length);
   }
+
+  public void getFloats(FloatBuffer receiver, float[] dst, int offset, int length) {
+    ensureValid();
+    receiver.get(dst, offset, length);
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
index d49f39f..f09e864 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBufferIndexInput.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
 import java.nio.LongBuffer;
 
 /**
@@ -33,6 +34,7 @@ import java.nio.LongBuffer;
  * </code>).
  */
 public abstract class ByteBufferIndexInput extends IndexInput implements RandomAccessInput {
+  private static final FloatBuffer EMPTY_FLOATBUFFER = FloatBuffer.allocate(0);
   private static final LongBuffer EMPTY_LONGBUFFER = LongBuffer.allocate(0);
 
   protected final long length;
@@ -44,6 +46,7 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
   protected int curBufIndex = -1;
   protected ByteBuffer curBuf; // redundant for speed: buffers[curBufIndex]
   private LongBuffer[] curLongBufferViews;
+  private FloatBuffer[] curFloatBufferViews;
 
   protected boolean isClone = false;
 
@@ -79,6 +82,7 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
   protected void setCurBuf(ByteBuffer curBuf) {
     this.curBuf = curBuf;
     curLongBufferViews = null;
+    curFloatBufferViews = null;
   }
 
   @Override
@@ -161,6 +165,36 @@ public abstract class ByteBufferIndexInput extends IndexInput implements RandomA
   }
 
   @Override
+  public final void readLEFloats(float[] floats, int offset, int len) throws IOException {
+    // See notes about readELongs above
+    if (curFloatBufferViews == null) {
+      curFloatBufferViews = new FloatBuffer[Float.BYTES];
+      for (int i = 0; i < Float.BYTES; ++i) {
+        // Compute a view for each possible alignment.
+        if (i < curBuf.limit()) {
+          ByteBuffer dup = curBuf.duplicate().order(ByteOrder.LITTLE_ENDIAN);
+          dup.position(i);
+          curFloatBufferViews[i] = dup.asFloatBuffer();
+        } else {
+          curFloatBufferViews[i] = EMPTY_FLOATBUFFER;
+        }
+      }
+    }
+    try {
+      final int position = curBuf.position();
+      FloatBuffer floatBuffer = curFloatBufferViews[position & 0x03];
+      floatBuffer.position(position >>> 2);
+      guard.getFloats(floatBuffer, floats, offset, len);
+      // if the above call succeeded, then we know the below sum cannot overflow
+      curBuf.position(position + (len << 2));
+    } catch (BufferUnderflowException e) {
+      super.readLEFloats(floats, offset, len);
+    } catch (NullPointerException npe) {
+      throw new AlreadyClosedException("Already closed: " + this);
+    }
+  }
+
+  @Override
   public final short readShort() throws IOException {
     try {
       return guard.getShort(curBuf);
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
index 5f49af0..5bf6b90 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDataInput.java
@@ -19,6 +19,8 @@ package org.apache.lucene.store;
 import java.io.EOFException;
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -34,6 +36,7 @@ import org.apache.lucene.util.RamUsageEstimator;
 public final class ByteBuffersDataInput extends DataInput
     implements Accountable, RandomAccessInput {
   private final ByteBuffer[] blocks;
+  private final FloatBuffer[] floatBuffers;
   private final int blockBits;
   private final int blockMask;
   private final long size;
@@ -50,7 +53,8 @@ public final class ByteBuffersDataInput extends DataInput
     ensureAssumptions(buffers);
 
     this.blocks = buffers.stream().map(buf -> buf.asReadOnlyBuffer()).toArray(ByteBuffer[]::new);
-
+    // pre-allocate this array and create the FloatBuffers lazily
+    this.floatBuffers = new FloatBuffer[blocks.length * Float.BYTES];
     if (blocks.length == 1) {
       this.blockBits = 32;
       this.blockMask = ~0;
@@ -197,6 +201,51 @@ public final class ByteBuffersDataInput extends DataInput
     }
   }
 
+  @Override
+  public void readLEFloats(float[] arr, int off, int len) throws EOFException {
+    try {
+      while (len > 0) {
+        FloatBuffer floatBuffer = getFloatBuffer(pos);
+        floatBuffer.position(blockOffset(pos) >> 2);
+        int chunk = Math.min(len, floatBuffer.remaining());
+        if (chunk == 0) {
+          // read a single float spanning the boundary between two buffers
+          arr[off] = Float.intBitsToFloat(Integer.reverseBytes(readInt(pos - offset)));
+          off++;
+          len--;
+          pos += Float.BYTES;
+          continue;
+        }
+
+        // Update pos early on for EOF detection, then try to get buffer content.
+        pos += chunk << 2;
+        floatBuffer.get(arr, off, chunk);
+
+        len -= chunk;
+        off += chunk;
+      }
+    } catch (BufferUnderflowException | IndexOutOfBoundsException e) {
+      if (pos - offset + Float.BYTES > size()) {
+        throw new EOFException();
+      } else {
+        throw e; // Something is wrong.
+      }
+    }
+  }
+
+  private FloatBuffer getFloatBuffer(long pos) {
+    // This creates a separate FloatBuffer for each observed combination of ByteBuffer/alignment
+    int bufferIndex = blockIndex(pos);
+    int alignment = (int) pos & 0x3;
+    int floatBufferIndex = bufferIndex * Float.BYTES + alignment;
+    if (floatBuffers[floatBufferIndex] == null) {
+      ByteBuffer dup = blocks[bufferIndex].duplicate();
+      dup.position(alignment);
+      floatBuffers[floatBufferIndex] = dup.order(ByteOrder.LITTLE_ENDIAN).asFloatBuffer();
+    }
+    return floatBuffers[floatBufferIndex];
+  }
+
   public long position() {
     return pos - offset;
   }
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
index 7a0b355..7a66211 100644
--- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersIndexInput.java
@@ -188,6 +188,12 @@ public final class ByteBuffersIndexInput extends IndexInput implements RandomAcc
   }
 
   @Override
+  public void readLEFloats(float[] floats, int offset, int len) throws IOException {
+    ensureOpen();
+    in.readLEFloats(floats, offset, len);
+  }
+
+  @Override
   public IndexInput clone() {
     ensureOpen();
     ByteBuffersIndexInput cloned =
diff --git a/lucene/core/src/java/org/apache/lucene/store/DataInput.java b/lucene/core/src/java/org/apache/lucene/store/DataInput.java
index b1203bb..dc3cf25 100644
--- a/lucene/core/src/java/org/apache/lucene/store/DataInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/DataInput.java
@@ -187,6 +187,20 @@ public abstract class DataInput implements Cloneable {
   }
 
   /**
+   * Reads a specified number of floats into an array at the specified offset.
+   *
+   * @param floats the array to read bytes into
+   * @param offset the offset in the array to start storing floats
+   * @param len the number of floats to read
+   */
+  public void readLEFloats(float[] floats, int offset, int len) throws IOException {
+    Objects.checkFromIndexSize(offset, len, floats.length);
+    for (int i = 0; i < len; i++) {
+      floats[offset + i] = Float.intBitsToFloat(Integer.reverseBytes(readInt()));
+    }
+  }
+
+  /**
    * Reads a long stored in variable-length format. Reads between one and nine bytes. Smaller values
    * take fewer bytes. Negative numbers are not supported.
    *
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java b/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
index 9bcc173..b664a45 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
@@ -439,4 +439,23 @@ public class TestMultiMMap extends BaseDirectoryTestCase {
       }
     }
   }
+
+  public void testLittleEndianFloatsCrossBoundary() throws Exception {
+    try (Directory dir = new MMapDirectory(createTempDir("testFloatsCrossBoundary"), 8)) {
+      try (IndexOutput out = dir.createOutput("Floats", newIOContext(random()))) {
+        out.writeByte((byte) 2);
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(3f)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(Float.MAX_VALUE)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(-3f)));
+      }
+      try (IndexInput input = dir.openInput("Floats", newIOContext(random()))) {
+        assertEquals(13, input.length());
+        assertEquals(2, input.readByte());
+        float[] ff = new float[4];
+        input.readLEFloats(ff, 1, 3);
+        assertArrayEquals(new float[] {0, 3f, Float.MAX_VALUE, -3f}, ff, 0);
+        assertEquals(13, input.getFilePointer());
+      }
+    }
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
index 3e42751..fe36545 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
@@ -235,6 +235,64 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
     }
   }
 
+  public void testAlignedFloats() throws Exception {
+    try (Directory dir = getDirectory(createTempDir("testAlignedFloats"))) {
+      try (IndexOutput out = dir.createOutput("Floats", newIOContext(random()))) {
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(3f)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(Float.MAX_VALUE)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(-3f)));
+      }
+      try (IndexInput input = dir.openInput("Floats", newIOContext(random()))) {
+        assertEquals(12, input.length());
+        float[] ff = new float[4];
+        input.readLEFloats(ff, 1, 3);
+        assertArrayEquals(new float[] {0, 3f, Float.MAX_VALUE, -3f}, ff, 0);
+        assertEquals(12, input.getFilePointer());
+      }
+    }
+  }
+
+  public void testUnalignedFloats() throws Exception {
+    int padding = random().nextInt(3) + 1;
+    try (Directory dir = getDirectory(createTempDir("testUnalignedFloats"))) {
+      try (IndexOutput out = dir.createOutput("Floats", newIOContext(random()))) {
+        for (int i = 0; i < padding; i++) {
+          out.writeByte((byte) 2);
+        }
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(3f)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(Float.MAX_VALUE)));
+        out.writeInt(Integer.reverseBytes(Float.floatToIntBits(-3f)));
+      }
+      try (IndexInput input = dir.openInput("Floats", newIOContext(random()))) {
+        assertEquals(12 + padding, input.length());
+        for (int i = 0; i < padding; i++) {
+          assertEquals(2, input.readByte());
+        }
+        float[] ff = new float[4];
+        input.readLEFloats(ff, 1, 3);
+        assertArrayEquals(new float[] {0, 3f, Float.MAX_VALUE, -3f}, ff, 0);
+        assertEquals(12 + padding, input.getFilePointer());
+      }
+    }
+  }
+
+  public void testFloatsUnderflow() throws Exception {
+    try (Directory dir = getDirectory(createTempDir("testFloatsUnderflow"))) {
+      final int offset = random().nextInt(4);
+      final int length = TestUtil.nextInt(random(), 1, 16);
+      try (IndexOutput out = dir.createOutput("Floats", newIOContext(random()))) {
+        byte[] b = new byte[offset + length * Float.BYTES - TestUtil.nextInt(random(), 1, Float.BYTES)];
+        random().nextBytes(b);
+        out.writeBytes(b, b.length);
+      }
+      try (IndexInput input = dir.openInput("Floats", newIOContext(random()))) {
+        input.seek(offset);
+        expectThrows(EOFException.class,
+            () -> input.readLEFloats(new float[length], 0, length));
+      }
+    }
+  }
+
   public void testString() throws Exception {
     try (Directory dir = getDirectory(createTempDir("testString"))) {
       IndexOutput output = dir.createOutput("string", newIOContext(random()));
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java
index f349129..fea8096 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexInputWrapper.java
@@ -155,6 +155,12 @@ public class MockIndexInputWrapper extends IndexInput {
   }
 
   @Override
+  public void readLEFloats(float[] floats, int offset, int len) throws IOException {
+    ensureOpen();
+    delegate.readLEFloats(floats, offset, len);
+  }
+
+  @Override
   public short readShort() throws IOException {
     ensureOpen();
     return delegate.readShort();