You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/11/24 10:54:08 UTC

[GitHub] [lucene-solr] jpountz commented on a change in pull request #2094: LUCENE-9047: Move the Directory APIs to be little endian

jpountz commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529369260



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
##########
@@ -570,6 +571,23 @@ static void writeCRC(IndexOutput output) throws IOException {
     if ((value & 0xFFFFFFFF00000000L) != 0) {
       throw new IllegalStateException("Illegal CRC-32 checksum: " + value + " (resource=" + output + ")");
     }
-    output.writeLong(value);
+    writeChecksum(output, value);
   }
+  
+  private static int readVersion(DataInput in) throws IOException {
+    return EndiannessReverserUtil.readInt(in);
+  }
+
+  private static void writeVersion(DataOutput out, int version) throws IOException {
+    EndiannessReverserUtil.writeInt(out, version);
+  }
+
+  private static long readChecksum(DataInput in) throws IOException {
+    return EndiannessReverserUtil.readLong(in);
+  }
+
+  private static void writeChecksum(DataOutput out, long checksum) throws IOException {
+    EndiannessReverserUtil.writeLong(out, checksum);
+  }

Review comment:
       My mental model for `EndiannessReverserUtil` is that after this PR is merged, we should look into no longer using it in current codecs. So maybe for CodecUtil we should avoid using it and directly do e.g. `long foo = Long.reverseBytes(in.readLong())`.

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortedNumericSortField.java
##########
@@ -106,20 +107,21 @@ public Provider() {
 
     @Override
     public SortField readSortField(DataInput in) throws IOException {

Review comment:
       likewise here

##########
File path: lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
##########
@@ -303,34 +306,37 @@ public static final SegmentInfos readCommit(Directory directory, String segmentF
   }
 
   /** Read the commit from the provided {@link ChecksumIndexInput}. */
-  public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException {
+  public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput inputCodec, long generation) throws IOException {
     Throwable priorE = null;
     int format = -1;
     try {
       // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
       // to read the magic ourselves.
-      int magic = input.readInt();
+      int magic = inputCodec.readInt();
       if (magic != CodecUtil.CODEC_MAGIC) {
-        throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
+        throw new IndexFormatTooOldException(inputCodec, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
       }
-      format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_70, VERSION_CURRENT);
+      format = CodecUtil.checkHeaderNoMagic(inputCodec, "segments", VERSION_70, VERSION_CURRENT);
       byte id[] = new byte[StringHelper.ID_LENGTH];
-      input.readBytes(id, 0, id.length);
-      CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
+      inputCodec.readBytes(id, 0, id.length);
+      CodecUtil.checkIndexHeaderSuffix(inputCodec, Long.toString(generation, Character.MAX_RADIX));
 
-      Version luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-      int indexCreatedVersion = input.readVInt();
+      Version luceneVersion = Version.fromBits(inputCodec.readVInt(), inputCodec.readVInt(), inputCodec.readVInt());
+      int indexCreatedVersion = inputCodec.readVInt();
       if (luceneVersion.major < indexCreatedVersion) {
         throw new CorruptIndexException("Creation version [" + indexCreatedVersion
-            + ".x] can't be greater than the version that wrote the segment infos: [" + luceneVersion + "]" , input);
+            + ".x] can't be greater than the version that wrote the segment infos: [" + luceneVersion + "]" , inputCodec);
       }
 
       if (indexCreatedVersion < Version.LATEST.major - 1) {
-        throw new IndexFormatTooOldException(input, "This index was initially created with Lucene "
+        throw new IndexFormatTooOldException(inputCodec, "This index was initially created with Lucene "
             + indexCreatedVersion + ".x while the current version is " + Version.LATEST
             + " and Lucene only supports reading the current and previous major versions.");
       }
 
+      // Wrap IndexInput for Big endian indexes
+      final DataInput input = format < VERSION_90 ? new EndiannessReverserIndexInput(inputCodec) : inputCodec;

Review comment:
       maybe move the below logic to a method so that we could do something like `return parseSegmentInfos( format < VERSION_90 ? new EndiannessReverserIndexInput(input) : input);` and avoid renaming `input` to `inptCodec` above. It would also reduce chances of mistakes as `input` and `inputCodec` would never be available at the same time.

##########
File path: lucene/core/src/java/org/apache/lucene/store/DataInput.java
##########
@@ -92,15 +92,20 @@ public void readBytes(byte[] b, int offset, int len, boolean useBuffer)
    * @see DataOutput#writeByte(byte)
    */
   public short readShort() throws IOException {
-    return (short) (((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF));
+    final byte b1 = readByte();
+    final byte b2 = readByte();
+    return (short) ((b2 << 8) | (b1 & 0xFF));
   }
 
   /** Reads four bytes and returns an int.
    * @see DataOutput#writeInt(int)
    */
   public int readInt() throws IOException {
-    return ((readByte() & 0xFF) << 24) | ((readByte() & 0xFF) << 16)
-         | ((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF);
+    final byte b1 = readByte();
+    final byte b2 = readByte();
+    final byte b3 = readByte();
+    final byte b4 = readByte();
+    return (b4 << 24) | (b3 & 0xFF) << 16 | (b2 & 0xFF) << 8 | (b1 & 0xFF);

Review comment:
       and likewise here for b4

##########
File path: lucene/core/src/java/org/apache/lucene/store/DataInput.java
##########
@@ -155,26 +160,25 @@ public int readZInt() throws IOException {
    * @see DataOutput#writeLong(long)
    */
   public long readLong() throws IOException {
-    return (((long)readInt()) << 32) | (readInt() & 0xFFFFFFFFL);
+    final byte b1 = readByte();
+    final byte b2 = readByte();
+    final byte b3 = readByte();
+    final byte b4 = readByte();
+    final byte b5 = readByte();
+    final byte b6 = readByte();
+    final byte b7 = readByte();
+    final byte b8 = readByte();
+    return ((b8 & 0xFFL) << 56) | (b7 & 0xFFL) << 48 | (b6 & 0xFFL) << 40 | (b5 & 0xFFL) << 32
+            | (b4 & 0xFFL) << 24 | (b3 & 0xFFL) << 16 | (b2 & 0xFFL) << 8 | (b1 & 0xFFL);

Review comment:
       any reason not to keep two calls to `readInt` instead of reading byte by byte?

##########
File path: lucene/core/src/java/org/apache/lucene/store/DataInput.java
##########
@@ -155,26 +160,25 @@ public int readZInt() throws IOException {
    * @see DataOutput#writeLong(long)
    */
   public long readLong() throws IOException {
-    return (((long)readInt()) << 32) | (readInt() & 0xFFFFFFFFL);
+    final byte b1 = readByte();
+    final byte b2 = readByte();
+    final byte b3 = readByte();
+    final byte b4 = readByte();
+    final byte b5 = readByte();
+    final byte b6 = readByte();
+    final byte b7 = readByte();
+    final byte b8 = readByte();
+    return ((b8 & 0xFFL) << 56) | (b7 & 0xFFL) << 48 | (b6 & 0xFFL) << 40 | (b5 & 0xFFL) << 32
+            | (b4 & 0xFFL) << 24 | (b3 & 0xFFL) << 16 | (b2 & 0xFFL) << 8 | (b1 & 0xFFL);
   }
 
   /**
-   * Read a specified number of longs with the little endian byte order.
-   * <p>This method can be used to read longs whose bytes have been
-   * {@link Long#reverseBytes reversed} at write time:
-   * <pre class="prettyprint">
-   * for (long l : longs) {
-   *   output.writeLong(Long.reverseBytes(l));
-   * }
-   * </pre>
-   * @lucene.experimental

Review comment:
       can you keep the experimental javadoc flag?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortedSetSortField.java
##########
@@ -86,8 +87,8 @@ public Provider() {
 
     @Override
     public SortField readSortField(DataInput in) throws IOException {

Review comment:
       and here again

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
##########
@@ -54,7 +54,10 @@ public void append(byte[] packedValue, int docID) throws IOException {
     assert closed == false : "Point writer is already closed";
     assert packedValue.length == config.packedBytesLength : "[packedValue] must have length [" + config.packedBytesLength + "] but was [" + packedValue.length + "]";
     out.writeBytes(packedValue, 0, packedValue.length);
-    out.writeInt(docID);
+    out.writeByte((byte) (docID >> 24));
+    out.writeByte((byte) (docID >> 16));
+    out.writeByte((byte) (docID >> 8));
+    out.writeByte((byte) (docID >> 0));

Review comment:
       oh tricky, is this because we need the serialized order to compare lexicographically? If so, can you leave a comment? And maybe do a `writeInt(Integer.reverseBytes(docID))` to keep it more readable?

##########
File path: lucene/core/src/java/org/apache/lucene/store/DataOutput.java
##########
@@ -210,8 +210,14 @@ public final void writeZInt(int i) throws IOException {
    * @see DataInput#readLong()
    */
   public void writeLong(long i) throws IOException {
-    writeInt((int) (i >> 32));
-    writeInt((int) i);
+    writeByte((byte) i);
+    writeByte((byte)(i >>  8));
+    writeByte((byte)(i >> 16));
+    writeByte((byte)(i >> 24));
+    writeByte((byte)(i >> 32));
+    writeByte((byte)(i >> 40));
+    writeByte((byte)(i >> 48));
+    writeByte((byte)(i >> 56));

Review comment:
       why not keep two calls to writeInt?

##########
File path: lucene/core/src/java/org/apache/lucene/search/SortField.java
##########
@@ -148,12 +149,12 @@ public Provider() {
 
     @Override
     public SortField readSortField(DataInput in) throws IOException {

Review comment:
       this one is tricky becase it's not versioned. Do you think we could remove all calls to `EndiannesReverserUtil` from it and instead fix callers to wrap the input with the EndiannessReverser wrapper?

##########
File path: lucene/core/src/java/org/apache/lucene/store/EndiannessReverserIndexInput.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.IOException;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * And {@link IndexInput} wrapper that changes the endianness of the provides
+ * index input.
+ *
+ * @lucene.internal
+ */
+public class EndiannessReverserIndexInput extends IndexInput {

Review comment:
       make it final?

##########
File path: lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
##########
@@ -96,11 +96,12 @@ public boolean visit() throws IOException {
         // Paranoia: make sure the primary node is not smoking crack, by somehow sending us an already corrupted file whose checksum (in its
         // footer) disagrees with reality:
         long actualChecksumIn = in.readLong();
-        if (actualChecksumIn != checksum) {
+        // CheckSum is written in Big Endian so we need to reverse bytes
+        if (actualChecksumIn != Long.reverseBytes(checksum)) {

Review comment:
       IMO we should instead fix the above call to do `long actualChecksumIn = Long.reverseBytes(in.readLong());` to get the actual checksum value? This way the below error message would also be correct?

##########
File path: lucene/core/src/java/org/apache/lucene/store/DataInput.java
##########
@@ -92,15 +92,20 @@ public void readBytes(byte[] b, int offset, int len, boolean useBuffer)
    * @see DataOutput#writeByte(byte)
    */
   public short readShort() throws IOException {
-    return (short) (((readByte() & 0xFF) <<  8) |  (readByte() & 0xFF));
+    final byte b1 = readByte();
+    final byte b2 = readByte();
+    return (short) ((b2 << 8) | (b1 & 0xFF));

Review comment:
       can you keep the conversion to an unsigned int of b2 like we had before?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org