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/23 11:09:28 UTC

[GitHub] [lucene-solr] iverase opened a new pull request #2094: LUCENE-9047: Move the Directory APIs to be little endian

iverase opened a new pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094


   Directory API is now little endian. Note that codecs still work on Big endian for backwards compatibility, therefore they reverse the bytes whenever they are writing / reading short, ints and longs.
   
   CodecUtils for header and footers has been modified to be little Indian. Still the version and checksum will be written / read reversing bytes for backwards compatibility.
   
   SegmentInfos is read / written in little endian, for previous version, the IndexInput is wrapped for backwards compatibility.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734302940


   > If you requested the reverse byte order it'd have to either wrap (slow) or return an optimized version (which in this case is simple too - clone the underlying buffer, switch byte order, return).
   
   I'm just suspicious we are only making things slower here, by explicitly swapping anything with java code. cpus since haswell have stuff like `movbe`, why should we support "slow" wrapping at all? I don't think slow should be possible here.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529539404



##########
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:
       yes, just need to make sure we revert bytes when writing the checksum again.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-733826001


   I had another iteration and add better backwards compatibility support to:
   
   * FST: I had to change slightly one test which I am not sure what is testing. All other test pass.
   * BlockTreeTermsReader
   * CompressingStoredFieldsReader
   * CompressingTermVectorsReader
   * FileIndexReader: This looks a bit strange as I store the version in one file (data file) but affects how we read the meta file.
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732172442


   That was my first approach but it become too hairy once I started to process headers and footers without wrapping the IndexOutput / IndexInput. One example is in the BKD tree we have the following line:
   
    https://github.com/apache/lucene-solr/blob/59b17366ff45d958810b1f8e4950eebd93f1b20d/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java#L993
   
   That means we will need a reference to the unwrapped IndexOutput to call this line. I did not want to change method signatures or move code around on this first pass so I went to manually revert endianness when needed so we could have a good understanding of the places where work is needed.
   
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732846144


   Sure, Adrien. Guys, don't get me wrong - I'm not vetoing the change, I'm just saying it looks terrible with all those calls all over the place. :) But I also can't sit on this now as my backlog is only increasing and Ignacio worked on a large patch to get this done so my complaints can be safely ignored... I'll just live with it.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732735040


   @jpountz @dweiss I tried again by wrapping the IndexOutput / IndexInput unsuccessfully. I think the reason is that in some cases we use temporary buffers via `ByteBuffersDataOutput#newResettableInstance` which I cannot wrap as the class is final. 


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734261088


   Perhaps I wasn't too clear but I think it'd be the same - for example, ByteBufferIndexInput would just implement ComplexTypeDataInput itself, by default assuming a certain byte order (LE or BE); the call to getReader with a matching byte order would return itself and it'd work identically as before. If you requested the reverse byte order it'd have to either wrap (slow) or return an optimized version (which in this case is simple too - clone the underlying buffer, switch byte order, return).


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732765973


   @dweiss  This should be only a temporary situation. The idea is to create new codecs that would not wrap those calls. The current codecs will be move to backwards codecs and only be used for indexes before 9.0.0.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529540139



##########
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:
       Just to keep my sanity :) done




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-735763595


   > what I understand is the files read by for example CompressingStoredFieldsReader, they hold versioning in the header?
   
   When we want to make changes to a file format we have two options:
    1. Either we create a new one and use it in a new codec, the old one only being used for bw compat.
    2. Or we handle this internally by incrementing the internal version of the file format.
   
   In general we lean towards 1 for the bigger changes and 2 for the smaller changes. For this change of endianness, we could decide to use option 1 across all file formats so that a given file format always knows what endianness it's supposed to use up-front.
   
   So in the example you mentionned, we could create a new Lucene90StoredFieldsFormat that doesn't share any logic with the current stored fields format and always writes and reads data in little endian order.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734320016


   The "slow" wrapper would be needed for the many IndexInput, IndexOutput implementations that don't have optimized versions and are part of the code at the moment. By "slow" I just mean plain Java code, not hardware-accelerated. For ByteBufferIndexInput (and perhaps a few other implementations) you could set the byte order on the underlying byte buffers and then delegate directly which, with some hope, would eventually get you to optimized native code. I'll tinker with this a bit, perhaps I can show a PoC that would be simpler to understand.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734221687


   As is frequently the case I don't have all the answers until I take a look at this... My feeling is that this "endianness" should apply at the lowest common denominator where complex-types are read (readint, readlong, etc.). If this is datainput/dataoutput then indeed they need this information when they're created (and it propagates up to where they're created). This is in a way similar to how bytebuffers are designed... only I'd be tempted to make this information constant for a datainput/dataoutput (so that source.order(newOrder) would create a delegate, not change the source's order).
   
   To determine the byte order of a file you either need an implicit knowledge (known codec version) or explicit knowledge (probing for some magic sequence). Whether this is always possible in Lucene - I can't tell without actually trying.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734260852


   I think there is plenty of opportunity to make performance regressions with this change, we should have benchmarks showing the improvement before we commit any such drastic changes to the low level interfaces?
   
   AFAIK, we already hacked away for postings-list decode to do what it needs in LE order for some vectorization reasons (LUCENE-9027).
   
   Otherwise, while its a nice goal, I am highly suspicious that the bswap is really slowing things down. Instead in practice the patch looks very complex, tons of swapping changes outside of backwards codecs (?), and even some proposals here that will definitely slow things down.
   
   So I'd like to see benchmark results before anything is committed.
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-737053701


   Hey. Just wanted to say I backtrack from the idea of having byte-order-conscious stream delegates. I still think it's a good idea and it'd provide a more efficient implementation than manually wrapping them with "reversing" wrappers but I don't have enough time to present a workable solution across all of Lucene codebase and I don't want to stall the effort here.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732177038


   If we only have a few cases like that, maybe we could fork the writeHeader/readHeader logic inside BKDWriter/BKDReader so that we can apply different migration rules to these calls than to `CodecUtil#readHeader/writeHeader`?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732975608


   I addressed Adrien's comments and introduce a `EndiannessReverserIndexOutput`. This allows to remove all the reverse dancing from the `SortFields` and those classes remain unchanged.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-741872764


   I changed PackedInts so they work on Little endian but still have the capability of reading data stored in big endian. This makes the PR even bigger but now we can run Lucene performance test directly. All test pass.
   
   I have run luceneutil wikimediall test and here are the results:
   
   ```
                       TaskQPS baseline      StdDevQPS my_modified_version      StdDev                Pct diff p-value
                     Fuzzy2       71.59      (8.0%)       70.39      (7.9%)   -1.7% ( -16% -   15%) 0.504
                     Fuzzy1       62.58      (6.0%)       62.10      (8.0%)   -0.8% ( -13% -   14%) 0.731
      BrowseMonthTaxoFacets        1.07      (3.8%)        1.06      (2.5%)   -0.7% (  -6% -    5%) 0.503
       BrowseDateTaxoFacets        1.01      (3.7%)        1.01      (2.2%)   -0.5% (  -6% -    5%) 0.631
                   PKLookup      200.93      (4.3%)      200.15      (3.7%)   -0.4% (  -8% -    7%) 0.759
   BrowseDayOfYearTaxoFacets        1.01      (3.7%)        1.00      (2.2%)   -0.4% (  -6% -    5%) 0.690
           HighSloppyPhrase        2.54      (4.2%)        2.53      (4.2%)   -0.3% (  -8% -    8%) 0.833
                LowSpanNear       20.41      (3.2%)       20.37      (3.7%)   -0.2% (  -6% -    6%) 0.846
                    Respell       42.35      (1.9%)       42.26      (1.6%)   -0.2% (  -3% -    3%) 0.702
       HighIntervalsOrdered        1.10      (1.6%)        1.10      (1.6%)   -0.2% (  -3% -    3%) 0.745
               OrHighNotMed      740.11      (3.5%)      738.98      (2.8%)   -0.2% (  -6% -    6%) 0.880
                    Prefix3       81.76      (5.3%)       81.71      (5.0%)   -0.1% (  -9% -   10%) 0.969
               HighSpanNear       13.40      (2.4%)       13.40      (2.8%)    0.0% (  -5% -    5%) 0.954
                    MedTerm     1483.15      (3.3%)     1484.29      (4.1%)    0.1% (  -7% -    7%) 0.947
              OrHighNotHigh      576.35      (4.8%)      577.11      (3.1%)    0.1% (  -7% -    8%) 0.917
                AndHighHigh       39.42      (1.8%)       39.48      (2.2%)    0.2% (  -3% -    4%) 0.805
      HighTermDayOfYearSort       68.23     (10.3%)       68.35     (10.2%)    0.2% ( -18% -   23%) 0.957
                MedSpanNear       77.71      (2.4%)       77.85      (2.6%)    0.2% (  -4% -    5%) 0.817
            MedSloppyPhrase       38.01      (2.1%)       38.09      (2.3%)    0.2% (  -4% -    4%) 0.754
                    LowTerm     1496.23      (2.8%)     1499.53      (3.6%)    0.2% (  -6% -    6%) 0.830
                   Wildcard       68.16      (8.0%)       68.32      (8.2%)    0.2% ( -14% -   17%) 0.930
                 AndHighMed       82.66      (2.6%)       82.96      (3.4%)    0.4% (  -5% -    6%) 0.707
      BrowseMonthSSDVFacets        4.15      (5.0%)        4.16      (5.2%)    0.4% (  -9% -   11%) 0.807
                  MedPhrase       17.85      (2.0%)       17.92      (2.0%)    0.4% (  -3% -    4%) 0.519
              OrNotHighHigh      638.37      (4.4%)      641.01      (3.8%)    0.4% (  -7% -    9%) 0.751
               OrNotHighMed      718.57      (5.1%)      721.55      (3.4%)    0.4% (  -7% -    9%) 0.764
            LowSloppyPhrase       21.19      (1.9%)       21.30      (2.1%)    0.5% (  -3% -    4%) 0.396
                 OrHighHigh       46.81      (1.6%)       47.06      (1.9%)    0.5% (  -2% -    4%) 0.343
               OrHighNotLow      776.11      (4.8%)      780.40      (3.4%)    0.6% (  -7% -    9%) 0.672
                  OrHighMed       38.36      (2.4%)       38.61      (2.9%)    0.7% (  -4% -    6%) 0.429
                 HighPhrase      423.52      (1.8%)      427.22      (1.9%)    0.9% (  -2% -    4%) 0.132
                  LowPhrase       56.12      (1.7%)       56.63      (2.1%)    0.9% (  -2% -    4%) 0.141
                  OrHighLow      326.11      (4.6%)      329.15      (6.4%)    0.9% (  -9% -   12%) 0.598
               OrNotHighLow      457.65      (3.3%)      462.47      (4.2%)    1.1% (  -6% -    8%) 0.376
                   HighTerm      930.64      (3.7%)      941.65      (4.2%)    1.2% (  -6% -    9%) 0.342
          HighTermMonthSort       41.85     (12.8%)       42.40     (12.5%)    1.3% ( -21% -   30%) 0.746
                 AndHighLow      461.07      (4.3%)      470.01      (5.8%)    1.9% (  -7% -   12%) 0.231
       HighTermTitleBDVSort       38.12     (12.0%)       38.93     (16.5%)    2.1% ( -23% -   34%) 0.641
                     IntNRQ       81.08      (1.6%)       82.85      (1.7%)    2.2% (  -1% -    5%) 0.000
   BrowseDayOfYearSSDVFacets        4.01      (0.7%)        4.12      (0.4%)    2.7% (   1% -    3%) 0.000
                 TermDTSort       60.83      (9.0%)       63.12     (11.6%)    3.8% ( -15% -   26%) 0.250
   ```
   
   It does not show any slowdowns.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732827520


   > But these methods are all over classes that are reused across codecs - not just codec-specific ones (CodecUtil, etc.)... once you commit this in something tells me they'll remain in the code for a loooong time...
   
   We will need to retain this until Lucene 11 indeed, though hopefully we should be able to move the vast majority of the code that swaps the byte order to `lucene/backward-codecs` in the near future as we migrate our file formats to the little endian byte order?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734138279


   The fst test verifies byte order of internal data serializer (reverse vs. normal byte order). 
   
   Every time I look at this diff I have a gut feeling that it should be possible to do this switch in a more transparent way... Even with all the temporary buffers we write to, etc.
   
   IndexInput and IndexOutput extend from DataInput/DataOutput. What if those classes were forced to carry the byte order with them (much like byte buffers do)? Initially this would mean all of DataInput/DataOutput we create would have an explicit big endian. We could then gradually switch these to LE for new codecs (in theory all the paired read/write of more complex types should still work) and eventually prohibit BE in DataInput/DataOutput constructor once all the backward codecs are gone...
   
   I am tempted to try out this approach this weekend, just out of curiosity. What do you think, @iverase ?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734255849


   I took a look out of curiosity. Here are my conclusions:
   
   - making DataInput/DataOutput know about the byte-order indeed introduces a lot of byteOrder-juggling around. I still think this is better than explicit byte-reversal wrappers because even if you need to store and pass byteOrder, you don't necessarily care about its value - it is just accepted and passed where it's consumed. It does add a lot of boilerplate, agreed.
   
   - while looking at all this I think the big design flaw is that streams opened by Directory instances are readily suitable for reading/ writing complex types... Many places that consume IndexInput/IndexOutput (or DataInput/DataOutput) are not using byte order sensitive methods at all. These places only read bytes, they wouldn't need to know about the byte order (making things much simpler). Whenever there is a need for readInt/readLong, etc. DataInput/DataOutput should be able to provide an efficient implementation depending on the byte order requested; something like:
   ```
   ComplexTypeDataOutput getWriter(ByteOrder bo);
   ComplexTypeDataInput getReader(ByteOrder bo);
   ```
   It doesn't have to be a wrapper object (but initially it can be - a common one for all DataInput/DataOutput implementations!) - the performance can be kept high in a specific implementation by the very source object implementing the byte order-sensitive interface in a specific way...
   
   What I see as a big upside of this is that any such refactoring should work for all the codecs - previous and current - by just passing the byte order to where it's relevant. You could even switch the byte order arbitrarily and (with the exception of backward-compatibility checks) all tests should still pass... Eventually, the byte order flag can be removed entirely as assertions are introduced in places where the byte order is fixed. The rest of the code would happily just use complex type read/write methods, oblivious of what the byte order in the underlying stream is.
   
   The disadvantage, of course, is that the very core Lucene API would have to change (DataInput, DataOutput, IndexInput, IndexOutput)...
   
   Do you think this makes sense? Or is Ignacio's patch a better way forward? 


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529537082



##########
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:
       Same as above




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732797761


   But these methods are all over classes that are reused across codecs - not just codec-specific ones (CodecUtil, etc.)... once you commit this in something tells me they'll remain in the code for a loooong time...


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-744450866


   Even though the API is not final, it looks like the incubating JDK vector API currently only supports little endian: https://github.com/openjdk/jdk/blob/9b5a2a6b24be140816cb40105c74fe85a5d1c04a/src/jdk.incubator.vector/share/classes/jdk/incubator/vector/Vector.java#L2944-L2955. This could be another reason to move file formats to little endian in order to be able to more easily leverage this vector API in the future?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529553632



##########
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:
       done




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-735732072


   > SegmentInfos certainly cannot know the endianness of the file up-front. But for other file formats, we could know this on a per-file-format basis? E.g. Lucene86PointsFormat always uses big endian but Lucene90PointsFormat will always use little endian?
   
   @jpountz That is true, but what I understand is the files read  by for example `CompressingStoredFieldsReader`, they hold versioning in the header?
   
   > So I'd like to see benchmark results before anything is committed.
   
   @rmuir I have created [JMH benchmarks](https://github.com/iverase/endianness_benchmark) that read longs using BytesBuffer and LongBuffer with different endianness. Results are here:
   
   ```
   Benchmark                          (byteOrder)   Mode  Cnt   Score   Error   Units
   ReadLongBenchmark.readBytesBuffer           LE  thrpt   25   9.015 ± 0.012  ops/us
   ReadLongBenchmark.readBytesBuffer           BE  thrpt   25   8.333 ± 0.040  ops/us
   ReadLongBenchmark.readLongsBuffer           LE  thrpt   25  24.510 ± 0.191  ops/us
   ReadLongBenchmark.readLongsBuffer           BE  thrpt   25   9.981 ± 0.034  ops/us
   ```


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732187741


   We have the same situation in PackedInts and FST where you cannot just pass a wrapped object.  
   There is as well the issue where the serialisation / deserialisation is endian dependent. In DocIdsWriter we serialise using:
   
   ```
             out.writeShort(out, (short) (docIds[start + i] >>> 8));
             out.writeByte((byte) docIds[start + i]);
   ```
   
   But deserialise using:
   
   ```
          long l1 = in readLong(in);
         long l2 = in readLong(in);
         long l3 = in readLong(in);
   ```
   
   There is a similar situation in `CompressingStoredFieldsWriter`. 
   
   I have another iteration to see if we can simplify but just wrapping the IndexOutput / IndexInput now that I have better understanding on the problem.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734208944


   I am working in the assumption that you don't know the Endianness of a file when you open it. Therefore I don't see how your approach can work except that the property in the DataInput / DataOutput is mutable?
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734258556


   > These places only read bytes, they wouldn't need to know about the byte order (making things much simpler).
   
   And slower in the case of mmap: which is our default implementation! How would readLong() work through and through with the mmap impl? Today it is one bounds check.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-737236259


   Thanks @dweiss! I think your approach is potentially more efficient but harder to make it to a state where you have everything working. I am currently taking a different approach by increasing the version number on the codec files. Therefore the writers should be  mostly untouched and only the readers should wrap the IndexInput when the version is lower that the current one.
   
   In most of the cases the real change on the codec is a one-liner. Unfortunately I need to do some refactor and therefore the patch is bigger. I opened an issue to do the refactor on the side as I think it is valuable even if this PR does not succeed.
   
   The only issue left is the PackedInts algorithms as I think the need to be adapted. I have done that already for the DirectWriter.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529537298



##########
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:
       yes




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732153572


   Wow, exciting! I'm curious why you didn't generalize usage of `EndiannessReverserIndexInput` to all index formats like you did for SegmentInfos. My expectation is that it would have helped keep the change contained to very few lines in the constructor of the various readers/writers rather than scattered in all places that read or write ints/longs?


----------------------------------------------------------------
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


[GitHub] [lucene-solr] iverase closed pull request #2094: LUCENE-9047: Move the Directory APIs to be little endian

Posted by GitBox <gi...@apache.org>.
iverase closed pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094


   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734754136


   I created a PR which kind of shows what I had in mind, it's here (won't compile, it's a PoC only to convey the idea).
   https://github.com/apache/lucene-solr/pull/2100
   
   The byte-order data type reader is acquired from DataInput (and subclasses) with a method that takes byte order as an argument.
   
   https://github.com/apache/lucene-solr/pull/2100/files#diff-f07fa6b85566fe26f4da99bae2f7b7ab416899c7de849ec924a8ac2e51e046a0R126
   
   This default method returns the byte-order-sensitive "typed" reader that implements delegates proper endianness and delegates to the source DataInput class. Subclasses can either reimplementing it fully if they can optimize byte-order access (as shown in ByteArrayDataInput) or can implement just one selected endianness (and the other one is provided via the wrapper).
   
   The same idea could be pulled up so that DataInput itself contains the byte-order, without any separate interface (TypedReader) but I think it's actually beneficial not to do so. In fact, a separate "typed reader" view makes it more elegant to delegate low-level methods to actual data source rather than overriding multiple methods from the superclass.
   
   This does seem like a lengthy effort but I wanted to show this to you for consideration.
   


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734223008


   Yes, exactly. And in those cases when you can't determine endianness from implicit information, some kind of probing will be required to determine it. Maybe you're right, @iverase that this forces the byte order property to be mutable - don't know this yet. I'd rather make it constant as it allows for easier code.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732970250


   @dweiss I don't think I had got you wrong, I wanted to make sure I had understood what you meant and that I was not missing a simplification for the backward-compatibility logic. This will certainly be a bit challenging to maintain, similarly to the surrogate dance we did when changing the order of the terms dictionary years ago. But I think it's still worth doing.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529536955



##########
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:
       I introduce a wrapper to the DataOutput as well and now this classes remain unchanged.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734218972


   SegmentInfos certainly cannot know the endianness of the file up-front. But for other file formats, we could know this on a per-file-format basis? E.g. `Lucene86PointsFormat` always uses big endian but `Lucene90PointsFormat` will always use little endian?


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-826650321


   superseded by https://github.com/apache/lucene/pull/107


-- 
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529539703



##########
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:
       done




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
iverase commented on a change in pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#discussion_r529536245



##########
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:
       yes, that make sense.




----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-734261626


   It's very much what RandomAccessInput interface is at the moment, btw.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732744204


   Thanks @iverase ! 
   
   I admit I grew up with big endian (assembly on M68k) and little endian always confused the hell out of me looking at memory hex dumps... but even my personal bias aside the patch you created scatters those reversing static method calls all over the place - this seems counterintuitive and against of what the goal of having little endian was (which is less code, fewer ops on LE architectures)?
   
   I may be naive but maybe if all DataOutput classes accepted ByteOrder (including ByteBuffersDataOutput) then the rest of the code could live with either byte order transparently on those paired writeints/ readints, etc? If these methods are not properly paired (like you pointed out is the case) then I'd try to fix that unpaired optimizations first... Otherwise it'll be mind-bending to follow what the code actually does.


----------------------------------------------------------------
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


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

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #2094:
URL: https://github.com/apache/lucene-solr/pull/2094#issuecomment-732161113


   I agree with Adrien - I thought (but please correct me if I'm wrong) that a single wrapper would be needed to keep the code compatible with existing indexes and dropping this wrapper would make everything work without those numerous calls to manual byte-shuffling in the "reverser"... I'm sorry if I fail to see the bigger picture here but looking at the diff it seems more complicated than it was 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