You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/01/24 22:45:18 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

RussellSpitzer opened a new pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966


   Closes #3960 


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794169307



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       I use byte in all the functions since I'm assuming in some engines we can get byte representations without a conversion (like in spark). The "size" is so that I don't have a float and double version




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794129573



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       Is the purpose of using `byte[]` so that we don't have a version for float and a version for double?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   Existing:
   | bits: l0, t1,  t0 |*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   With padding:
   | bits:l l1, t1, l0, t0 |0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800853640



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);

Review comment:
       The encoding must be fixed, and it must be UTF-8.
   
   Also byte order will match string/varchar order only if bytes are treated as unsigned.
   I guess he unsignedness applies to other types, so maybe it's documented already, i didn't notice.

##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());

Review comment:
       `val.length()` is not "encoding-aware". Maybe add a comment that it doesn't matter.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802099276



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());

Review comment:
       OH sorry I thought you were talking about something else, yeah let me switch this to the byte length of the encoded string




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800252771



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {

Review comment:
       I think it's much easier to read and reason about algorithms like this if you don't use the return value of ++ or -- operators.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800330107



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       (strings are always expected to be padded to a fixed length)




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801576560



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());

Review comment:
       Actually, don't add a comment. We must take byte-length into account, otherwise we would be truncating short non-all-ASCII string values.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800254390



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {
+            sourceByte++;
+            sourceBit = 7;
+          }
+        }
+      } while (columnsBinary[sourceColumn].length <= sourceByte && interleaveByte < interleavedSize);

Review comment:
       I don't understand how the `columnsBinary[sourceColumn].length <= sourceByte` loop works.
   
   I understand the block and I think that correctly moves to the next bit to be consumed by the main loop. But I don't see why the block is performed in a loop until `sourceByte` is beyond the length of one of the columns. Won't that cause this to continue consuming source bytes until they're exhausted and never stop to interleave them?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801184895



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{

Review comment:
       +1 to asking for a specific interleaved size. You could also have versions that produce int or long if you only want 4 or 8 bytes. That would work really well with Spark because you wouldn't have to store this in the variable-length part of UnsafeRow.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802098344



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {

Review comment:
       Will do




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802060982



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {

Review comment:
       It should be documented somewhere the produced byte[] should be compared lexicographically, with bytes as *unsigned*.

##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {

Review comment:
       How is the `length` going to be calculated in practice?
   Is it going to be a fixed prefix, like 128 or something?

##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());

Review comment:
       ... an example would be String "ą", single character, so length=1.
   We would copy only the first byte of the two-byte sequence (0xC4 0x85) for the letter.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800250355



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);

Review comment:
       I'm okay with this if you want to get this through quickly, but in the long term I think we should avoid the allocation cost. What I'd recommend since this is a util method is to add a `reuse` argument:
   
   ```java
     // this should probably live in the ByteBuffers util class
     public static ByteBuffer reuseOrAllocate(ByteBuffer reuse, int length) {
       if (reuse.hasArray() && reuse.arrayOffset() == 0 && reuse.capacity() == length) {
         reuse.position(0);
         reuse.limit(length);
         return reuse;
       } else {
         return ByteBuffer.allocate(length);
       }
     }
   
     public static ByteBuffer intToOrderedBytes(int val, ByteBuffer reused) {
       ByteBuffer bytes = reuseOrAllocate(reused, 4);
       bytes.putInt(val ^ 0x80000000);
       return bytes;
     }
   ```
   
   Then the caller can always call `ByteBuffer.array()` to get the underlying bytes, but it can keep reusing the space:
   ```java
     v1Bytes = intToOrderedBytes(v1, v1Bytes);
     v2Bytes = intToOrderedBytes(v2, v2Bytes);
     zorderBytes = zorder(v1Bytes, v2Bytes, zorderBytes);
   ```




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800750480



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{

Review comment:
       My base implementation of this for spark just uses a fixed byte size for strings since each column has to contribute the same number of bytes every time. I put this responsibility in the toOrderedBytes function and not in the interleave itself. We need all entries from the same column to be the same length for the following situation
   
   Imagine columns A,B, and C where A is between 0 and 2 bits, but B and C are always 2 bits. You could get ZValues like
   
   (AA, BB, CC) - >ABCABC
   (A_, BB, CC) - >ABCBC
   (__, BB, CC) -> BCBC
   
   The sorting is now dependent on the length of the column value and not the value itself. Shorter A's being clustered differently than longer A's.
   
   As for in the trailing situation, I think it's worthwhile to keep those bits as well since we can just use them for hierarchal sort within clusters. You can imagine the interleave bits creating multidimensional clustered groups and the final bits producing a hierarchal sort within those groups based on the unshared portion of the longest element.
   
   I do think in the future we may want to save space here and then we could trim the max length of a contributed byte array to the length of the second longest contributor. 




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800948110



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty

Review comment:
       I'll fix up this comment, it's pretty unclear at the moment




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795294696



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       Or Do something inside catalyst ... which also may be a better option




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795285630



##########
File path: build.gradle
##########
@@ -222,6 +222,7 @@ project(':iceberg-core') {
     }
 
     testImplementation "org.xerial:sqlite-jdbc"
+    testImplementation "org.apache.commons:commons-lang3"

Review comment:
       You might want to just update our internal `RandomUtil` class instead. It's pretty close to doing what you want: https://github.com/apache/iceberg/blob/master/api/src/test/java/org/apache/iceberg/util/RandomUtil.java#L200-L209




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r793303020



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {

Review comment:
       maybe documenting the assumption that NaN values are normalized to Double.NaN




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795853721



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       I think that Catalyst should do okay with this because it uses Unsafe to pull values out of memory. I believe that happens by just treating an offset like its already a long or double, so it isn't doing a byte-wise operation. And for testing that also works better with GenericInternalRow, which actually stores the JVM types.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r796050017



##########
File path: build.gradle
##########
@@ -222,6 +222,7 @@ project(':iceberg-core') {
     }
 
     testImplementation "org.xerial:sqlite-jdbc"
+    testImplementation "org.apache.commons:commons-lang3"

Review comment:
       Are you ok with me changing the RandomUtil Class to actually do unicode? I think that's what we generally need 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800254976



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {
+            sourceByte++;
+            sourceBit = 7;
+          }
+        }
+      } while (columnsBinary[sourceColumn].length <= sourceByte && interleaveByte < interleavedSize);

Review comment:
       Oh, nevermind. I get it. It runs the loop again if the column has been exhausted. That is, when the column's length is less than or equal to the source byte position. Otherwise it exits.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801077334



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {
+      // The signed magnitude is positive set the first bit (reversing the sign so positives order after negatives)
+      floatBytes[0] = (byte) (floatBytes[0] | (1 << 7));
+    } else {
+      // The signed magnitude is negative so flip the first bit (reversing the sign so positives order after negatives)
+      // Then flip all remaining bits so numbers with greater negative magnitude come before those
+      // with less magnitude (reverse the order)
+      for (int i = 0; i < floatBytes.length; i++) {
+        floatBytes[i] = (byte) ~floatBytes[i];
+      }
+    }
+    return floatBytes;
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] orderUTF8LikeBytes(byte[] stringBytes, int size) {
+    if (stringBytes == null) {
+      return new byte[size];
+    }
+    return Arrays.copyOf(stringBytes, size);
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      if (--interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        if (++sourceColumn == columnsBinary.length) {

Review comment:
       Removed and added more clear comments about what we are doing in this algorithm




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801087518



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{

Review comment:
       One thing you could do since it doesn't matter is just break out of the interleave loop if moving to the next column doesn't change the column index. It would be nice to not allocate the space as well.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801078298



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {

Review comment:
       Changed the number of trials run so now this takes a few hundred ms. The longest test in the suite now is the String compare test which is around 1 second on my 2019 MBP




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802249270



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {

Review comment:
       In the our Spark implementation (see the follow up to this pr) we just set a length for the moment. Mostly to save us from unbounded strings increasing the size of our z-order algorithm and making sure the column always contributes the same number of bytes. 
   
   In the future I hope we can use statistics to do something smarter, or maybe not even use truncation and use some binning function or something like that based on the range of values in the column.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802248797



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {

Review comment:
       Yep, I will add this




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800308967



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       > Result is going to be: 0b1010101010101010101010101010101011111111. All values of v1 interleave with just the upper bits of v2. Is that the intended behavior?
   
   nit: I think the result fails to account for `*ToBytes` calls which flip the first leading bits.  Which should make the byte ordering similar to strings
   
   > This may be handy when trying to handle magnitude problems, assuming that the magnitude is always reflected in the type. But you could easily have a case where v1 and v2 are both in the range of 0x0000 to 0xFFFF and then this behavior would not actually produce a useful z-order.
   
   Not sure if the nit above addresses your concerns and what exactly you mean by not a useful z-order, could you expand on them?  My understanding is that as long as the input bytes that get interleaved have an equivalent lexicographic order to the original inputs it is a z-order and should maintain the clustering properties z-orders provide, if you then compare the output bytes lexicographically.   The only other principled option I could see here would be to zero pad all numeric values to the same number of bytes, but without thinking too deeply about it I think this just takes extra space.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800251862



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |

Review comment:
       Why not use `|=` instead of adding this every 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800730941



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       We have been talking about this a bit, but I think the only way to really solve this is to know the range of all column values before getting their binary values and shifting then. 
   
   I think at this point in the algorithm we don't have enough information to know whether we can bit shift a value into higher order bits but I do agree eventually we should have a representation that packs the information into the highest bits possible. Ideally I think this is a responsibility of the "toOrderedBinary" which I would like to have a "min/max" parameters in the future so we can shift and tighten the domain if possible.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800257166



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{

Review comment:
       You could argue that the last byte of `test[0]` should be ignored. Would it be better to do that so that you can zorder strings and know that you'll get just the prefix? Or do you think that this should be configured with a byte range in the strings?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   Existing:
   | bits: l<sub>0</sub>, t<sub>1</sub>,  t<sub>0</sub> |*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   With padding:
   | bits: l<sub>1</sub>, t<sub>1</sub>, l<sub>0</sub>, t<sub>0</sub> |0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800251637



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {

Review comment:
       If you update this to accept a reused buffer, then we need to remember to zero out the bytes.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802098213



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {

Review comment:
       The debate we were having before was wether to limit it to the max common length of columns, or whether to let it go beyond that.
   
   Like with (A, B, CC, DDD)
   
   Do you return
   
   A. : All bytes that are available
   ABCDCDD 
   
   B. : All bytes that can be interleaved with at least one other column
   ABCDCD
   
   Or
   
   C. : All bytes that can be interleaved with all other columns
   ABCD
   
   
   My current implementation in Spark just does A which is the sum of all column lengths, but we could do B and save some space at the cost of losing a bit of single column ordering. I don't think C actually makes a lot of sense unless we do some hind of bin hashing and actually generate bytes all of the same size.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r805085011



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       @findepi, I see your point. If you still want the string ordering then you have to preserve the string. You're right about that. I was focusing on interleaving the ordering across the columns, but it's of course necessary to still sort by the remaining bytes in the longest column!




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794127868



##########
File path: build.gradle
##########
@@ -222,6 +222,7 @@ project(':iceberg-core') {
     }
 
     testImplementation "org.xerial:sqlite-jdbc"
+    testImplementation "org.apache.commons:commons-lang3"

Review comment:
       What is this used for? We don't usually depend on Apache commons stuff.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r793251873



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {

Review comment:
       not sure if it is worth handling, but can NaN's on the JVM have different representations and do you want to ensure they are sorted together?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795287567



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       We should make sure that assumption holds because converting floats and doubles is probably much faster if we use full values rather than individual bytes. The [HBase OrderedBytes](https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java#L1482) implementation for double is this:
   
   ```java
     long asLong = Double.doubleToLongBits(doubleValue);
     asLong ^= ((asLong >> (Long.SIZE - 1)) | Long.MIN_VALUE);
   ```
   
   That avoids looping over the bytes and avoids an if statement in a tight loop by producing the correct value to XOR with to optionally the non-sign flip bits or not. That is probably much faster than fetching and flipping the bits of each byte individually, if we are starting off with a float or double.
   
   Also, `doubleToLongBits` will normalize NaN values for you.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r804384970



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       > cast the integer in your example to a bigint
   
   that works, for numbers at leasts
   
   > The remainder of the 50 bytes after the first 8 doesn't matter.
   
   It does:
   
   `z_order(a_short, z_order_interleaving)` -- it's unlikely that 2 first bytes of a string suffice for anything useful (`ht` for urls)
   
   unless it's covered somewhere else:
   if you `order by z_order_interleaving, c1, c2, c3`, then it `z_order_interleaving` can be shorter
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800252198



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);

Review comment:
       Shouldn't this use `>>> sourceBit` so that the sign bit isn't propagated? or are we guaranteed that won't happen because this gets converted into an int so it is too sparse to worry?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800730941



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       We have been talking about this a bit, but I think the only way to really solve this is to know the range of all column values before getting their binary values and shifting then. 
   
   I think at this point in the algorithm (during interleaving) we don't have enough information to know whether we can bit shift a value into higher order bits but I do agree eventually we should have a representation that packs the information into the highest bits possible. Ideally I think this is a responsibility of the "toOrderedBinary" which I would like to have a "min/max" parameters in the future so we can shift and tighten the domain if possible.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800996563



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       > I think the result fails to account for *ToBytes calls which flip the first leading bits. Which should make the byte ordering similar to strings IIUC
   
   Yes, I didn't do the sign bit flip, but the result is pretty much the same. All of the relevant bits of the short are interleaved with the irrelevant bits of the int and you effectively get an expensive way to sort by the short and then by the int.
   
   I think it makes the most sense to cast everything to the widest type. I think that is the behavior most people would expect: if you zorder a short with an int, it happens by magnitude not bit position.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801084085



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       There's another simple option that I think solves an additional challenge...
   
   I'd propose that we cast everything to float so we don't have to worry about choosing a type and converting everything to it. That also gives us a more compact representation that is better for packing multiple columns. That's because floating point values are represented as scientific notation with base-2. The exponent encodes how many positions the fraction was shifted so that its first digit is a 1. Alternatively, you can think of it as encoding how many bits at the start of the number are 0, then storing a normalized fraction.
   
   That effectively compresses the most significant parts of a number into the highest bits. For float32, there's a sign bit, 8 bits of exponent, and 23 bits of fraction (mantissa).
   
   Converting all the values to floating point would solve the immediate problem because all numbers would use the same representation, giving what I think is the expected behavior. But it also means that we can fit more columns in 32 bits or 64 bits, which are going to be much faster for comparison once we pass them off to Spark.
   
   For example, if we took 4 integers and interleaved them, it would take 128 bits. But if we convert those to float first, then the most significant parts can fit in half that: 16 bits per value can carry the sign bit, the full 8-bit exponent, and 7 bits of the fraction. That's about 128 partitions of the number space per exponent value.
   
   If you don't want to lose so much of the value, then fewer columns do better. And we can always fall back to producing the full byte array if we want.
   
   What do you guys think?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#issuecomment-1031948718


   Thanks everyone for taking a look, I tried to take into account the major points and I know we have lots of room for research and improvement in the future. Please let me know if you have any more feedback that we should address for the initial version of the feature


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801839954



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);

Review comment:
       I think @findepi has a point about length. In addition, this call to `getBytes` is going to allocate a byte array. It doesn't help to reuse a buffer if we are going to do an allocation anyway.
   
   One thing you can do is to use [`CharsetEncoder.encode(CharBuffer, ByteBuffer, boolean)`](https://docs.oracle.com/javase/8/docs/api/java/nio/charset/CharsetEncoder.html#encode-java.nio.CharBuffer-java.nio.ByteBuffer-boolean-). That way, you'd get a `CharBuffer` wrapping the original string without reallocating and encode into the buffer that's passed in. You'd also get information about the result, but you probably don't really care what happens. Underflow means the string is short and overflow means that you ran out of space. Neither one really affects zorder much.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801839954



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);

Review comment:
       I think @findepi has a point about length. In addition, this call to `getBytes` is going to allocate a byte array. It doesn't help to reuse a buffer if we are going to do an allocation anyway.
   
   One thing you can do is to use `CharsetEncoder.encode(CharBuffer, ByteBuffer, boolean)`. That way, you'd get a `CharBuffer` wrapping the original string without reallocating and encode into the buffer that's passed in. You'd also get information about the result, but you probably don't really care what happens. Underflow means the string is short and overflow means that you ran out of space. Neither one really affects zorder much.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800308967



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       > Result is going to be: 0b1010101010101010101010101010101011111111. All values of v1 interleave with just the upper bits of v2. Is that the intended behavior?
   
   nit: I think the result fails to account for `*ToBytes` calls which flip the first leading bits.  Which should make the byte ordering similar to strings IIUC
   
   > This may be handy when trying to handle magnitude problems, assuming that the magnitude is always reflected in the type. But you could easily have a case where v1 and v2 are both in the range of 0x0000 to 0xFFFF and then this behavior would not actually produce a useful z-order.
   
   Not sure if the nit above addresses your concerns and what exactly you mean by not a useful z-order, could you expand on them?  My understanding is that as long as the input bytes that get interleaved have an equivalent lexicographic order to the original inputs it is a z-order and should maintain the clustering properties z-orders provide, if you then compare the output bytes lexicographically.   The only other principled option I could see here would be to zero pad all numeric values to the same number of bytes, but without thinking too deeply about it I think this just takes extra space.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801828625



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -108,54 +108,70 @@ private ZOrderByteUtils() {
    * Strings are lexicographically sortable BUT if different byte array lengths will
    * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
    * This implementation just uses a set size to for all output byte representations. Truncating longer strings
-   * and right padding 0 for shorter strings.
+   * and right padding 0 for shorter strings. Requires UTF8 (or ASCII) encoding for ordering guarantees to hold.

Review comment:
       I agree. We should ensure that we use UTF-8 to convert to bytes.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800255169



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{
+        OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII,
+        IOIOIOIO, IOIOIOIO,
+        OIOIOIOI, OIOIOIOI,
+        OOOOIIII};
+    Assert.assertArrayEquals("Should combine mixed byte arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testIntOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      int aInt = random.nextInt();
+      int bInt = random.nextInt();
+      int intCompare = Integer.signum(Integer.compare(aInt, bInt));
+      byte[] aBytes = ZOrderByteUtils.intToOrderedBytes(aInt);
+      byte[] bBytes = ZOrderByteUtils.intToOrderedBytes(bInt);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aInt, bInt, intCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          intCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testLongOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      long aLong = random.nextInt();
+      long bLong = random.nextInt();
+      int longCompare = Integer.signum(Long.compare(aLong, bLong));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aLong);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bLong);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aLong, bLong, longCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          longCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testShortOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      short aShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      short bShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      int longCompare = Integer.signum(Long.compare(aShort, bShort));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aShort);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bShort);

Review comment:
       +1




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801135964



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{

Review comment:
       I think a good way to do this may be to change the api slightly to
   
   ```java
     public static byte[] interleaveBits(byte[][] columnsBinary) {
       return interleaveBits(columnsBinary,
           Arrays.stream(columnsBinary).mapToInt(column -> column.length).max().getAsInt());
     }
   
     /**
      * Interleave bits using a naive loop. Variable length inputs are allowed but to get a consistent ordering it is
      * required that every column contribute the same number of bytes in each invocation. Bits are interleaved from all
      * columns that have a bit available at that position. Once a Column has no more bits to produce it is skipped in the
      * interleaving.
      * @param columnsBinary an array of ordered byte representations of the columns being ZOrdered
      * @param interleavedSize the number of bytes to use in the output
      * @return the columnbytes interleaved
      */
     public static byte[] interleaveBits(byte[][] columnsBinary, int interleavedSize) {
     ```
     
     So that instead of calculating the output-size, the caller can just pass in the number of bytes it wants out of the algorithm. This handles bailing out earlier as well as not allocating as much. I can leave the original signature for the test suite.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800748901



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       > I think at this point in the algorithm (during interleaving) we don't have enough information to know whether we can bit shift a value into higher order bits but I do agree eventually we should have a representation that packs the information into the highest bits possible. Ideally I think this is a responsibility of the "toOrderedBinary" which I would like to have a "min/max" parameters in the future so we can shift and tighten the domain if possible.
   
   I agree.  I think this might imply that we always want an  equal number of bits on each input to interleave (need to think about this some more)?  I guess falling back to traditional sorting isn't a bad fallback but we are wasting compute to make that happen.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800706563



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by follow).  The top and outer left most columns represent the input numbers
   
   ||*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   ||0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as 2)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r803094872



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       @findepi, while those do have different lengths, is it valuable to interleave them directly? We had a discussion above about interleaving short and int. I think that the behavior most people would expect is to interleave by numeric magnitude, not the actual bytes. Basically, if two columns have the same numeric distribution, they should interleave well regardless of the type widths.
   
   If you agree there, then what you'd do is cast the integer in your example to a bigint. Then you have 2 8-byte buffers and 1 50-byte buffer. The remainder of the 50 bytes after the first 8 doesn't matter.
   
   There are cases where you may want to use different lengths, but to me you should be calling the version with a specified length for those.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794139908



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {
+      // The signed magnitude is positive set the first bit (reversing the sign so positives order after negatives)
+      floatBytes[0] = (byte) (floatBytes[0] | (1 << 7));
+    } else {
+      // The signed magnitude is negative so flip the first bit (reversing the sign so positives order after negatives)
+      // Then flip all remaining bits so numbers with greater negative magnitude come before those
+      // with less magnitude (reverse the order)
+      for (int i = 0; i < floatBytes.length; i++) {
+        floatBytes[i] = (byte) ~floatBytes[i];
+      }
+    }
+    return floatBytes;
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] orderUTF8LikeBytes(byte[] stringBytes, int size) {
+    if (stringBytes == null) {
+      return new byte[size];
+    }
+    return Arrays.copyOf(stringBytes, size);
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      if (--interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        if (++sourceColumn == columnsBinary.length) {

Review comment:
       I think it's usually a bad idea to use the output value of a ++ or -- operator. It's just hard to understand what's going on and I think prone to errors.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795309778



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       Actually I think I can still do this with duds for the prototype, Will code all this up Monday
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r796127675



##########
File path: build.gradle
##########
@@ -222,6 +222,7 @@ project(':iceberg-core') {
     }
 
     testImplementation "org.xerial:sqlite-jdbc"
+    testImplementation "org.apache.commons:commons-lang3"

Review comment:
       actually i'll just use the internal class for now. work on that later
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] nastra commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
nastra commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r798844481



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty

Review comment:
       nit: is the `none of which are empty` commit still valid given that there are tests for empty arrays?

##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{
+        OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII,
+        IOIOIOIO, IOIOIOIO,
+        OIOIOIOI, OIOIOIOI,
+        OOOOIIII};
+    Assert.assertArrayEquals("Should combine mixed byte arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testIntOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      int aInt = random.nextInt();
+      int bInt = random.nextInt();
+      int intCompare = Integer.signum(Integer.compare(aInt, bInt));
+      byte[] aBytes = ZOrderByteUtils.intToOrderedBytes(aInt);
+      byte[] bBytes = ZOrderByteUtils.intToOrderedBytes(bInt);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aInt, bInt, intCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          intCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testLongOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      long aLong = random.nextInt();
+      long bLong = random.nextInt();
+      int longCompare = Integer.signum(Long.compare(aLong, bLong));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aLong);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bLong);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aLong, bLong, longCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          longCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testShortOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      short aShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      short bShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      int longCompare = Integer.signum(Long.compare(aShort, bShort));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aShort);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bShort);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+              "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+              aShort, bShort, longCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          longCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testTinyOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      long aByte = (byte) (random.nextInt() % (Byte.MAX_VALUE + 1));
+      long bByte = (byte) (random.nextInt() % (Byte.MAX_VALUE + 1));
+      int longCompare = Integer.signum(Long.compare(aByte, bByte));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aByte);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bByte);

Review comment:
       I think those 2 should be `tinyintToOrderedBytes`

##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{
+        OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII,
+        IOIOIOIO, IOIOIOIO,
+        OIOIOIOI, OIOIOIOI,
+        OOOOIIII};
+    Assert.assertArrayEquals("Should combine mixed byte arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testIntOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      int aInt = random.nextInt();
+      int bInt = random.nextInt();
+      int intCompare = Integer.signum(Integer.compare(aInt, bInt));
+      byte[] aBytes = ZOrderByteUtils.intToOrderedBytes(aInt);
+      byte[] bBytes = ZOrderByteUtils.intToOrderedBytes(bInt);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aInt, bInt, intCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          intCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testLongOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      long aLong = random.nextInt();
+      long bLong = random.nextInt();
+      int longCompare = Integer.signum(Long.compare(aLong, bLong));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aLong);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bLong);
+      int byteCompare = Integer.signum(UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes));
+
+      Assert.assertEquals(String.format(
+          "Ordering of longs should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aLong, bLong, longCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          longCompare, byteCompare);
+    }
+  }
+
+  @Test
+  public void testShortOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      short aShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      short bShort = (short) (random.nextInt() % (Short.MAX_VALUE + 1));
+      int longCompare = Integer.signum(Long.compare(aShort, bShort));
+      byte[] aBytes = ZOrderByteUtils.longToOrderBytes(aShort);
+      byte[] bBytes = ZOrderByteUtils.longToOrderBytes(bShort);

Review comment:
       I think those 2 should both be `shortToOrderBytes`

##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {

Review comment:
       takes 17-18 seconds to run on my local machine, just wondering whether we want to have such long running unit tests




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794170748



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {
+      // The signed magnitude is positive set the first bit (reversing the sign so positives order after negatives)
+      floatBytes[0] = (byte) (floatBytes[0] | (1 << 7));
+    } else {
+      // The signed magnitude is negative so flip the first bit (reversing the sign so positives order after negatives)
+      // Then flip all remaining bits so numbers with greater negative magnitude come before those
+      // with less magnitude (reverse the order)
+      for (int i = 0; i < floatBytes.length; i++) {
+        floatBytes[i] = (byte) ~floatBytes[i];
+      }
+    }
+    return floatBytes;
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] orderUTF8LikeBytes(byte[] stringBytes, int size) {
+    if (stringBytes == null) {
+      return new byte[size];
+    }
+    return Arrays.copyOf(stringBytes, size);
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {

Review comment:
       I think that would be a good thing for a future improvement. If you see the Spark request has some benchmarks and even with wrapping all these functions in UDFS and applying them to rows that way, it's still only about 2~3 x slower than sort with as many expressions. So I think the perf is probably ok to start with.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801830044



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();

Review comment:
       Can we return the ByteBuffer instead since we're passing in a ByteBuffer?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801840261



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       Should we only allow this when all the columns have the same length?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801859352



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       I don't think so? Shouldn't we be allowing columns to contribute different amounts of bytes? 
   
   This is one of the difficulties for working with strings and other types since not only are we going to need more bytes for strings, but their most significant bytes may be rather late in the string.
   
   For example
   
   "www.iceberg.org/index"
   "www.iceberg.org/spark"
   
   Only have a differing byte at position 17.
   
   This is one of my reluctances to require similar contributions of byte sizes until we have a better "toUnorderedBytes" function.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800752068



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);

Review comment:
       I think this is worthwhile to get in now, it isn't a huge change and should let us think a bit more about the Spark implementation's memory efficiency 




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800256915



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {

Review comment:
       I'd probably `@Ignore` this test, but it's good to see it working.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800755604



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);

Review comment:
       I actually haven't thought this through, This is one of the reasons I added the big fuzzing test of the interleave code since I have a hard time thinking through byte path in my head. I can switch to >>> though since I think that is correct




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800853640



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);

Review comment:
       The encoding must be fixed, and it must be UTF-8, if we want the ordering of the produced bytes be coherent with value ordering.
   
   Also byte order will match string/varchar order only if bytes are treated as unsigned.
   I guess he unsignedness applies to other types, so maybe it's documented already, i didn't notice.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800256990



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       I'm not sure that this is the correct behavior in all cases.
   
   I think that this is the correct behavior for strings because the length of the string doesn't matter. `a` is always less than `bbb`. However, this is not the right behavior for numbers. For example:
   
   ```java
   short v1 = 0xFFFF
   int v2 = 0x0000FFFF
   byte[] result = ZOrderByteUtils.interleaveBits(new byte[] { shortToOrderedBytes(v1), intToOrderedBytes(v2) }
   ```
   
   Result is going to be: `0b1010101010101010101010101010101011111111`. All values of v1 interleave with just the upper bits of v2. Is that the intended behavior?
   
   This may be handy when trying to handle magnitude problems, assuming that the magnitude is always reflected in the type. But you could easily have a case where v1 and v2 are both in the range of `0x0000` to `0xFFFF` and then this behavior would not actually produce a useful zorder.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800753064



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {

Review comment:
       I can cut down the number of example :) I just really wanted to have some kind of fuzzing since I know we are going to want to optimize the interleave example with fancier byte tricks 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801053113



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       I have the general feeling that different columns sharing the same magnitude values but different types is more of a special case but I don't have any empirical evidence for that. I'm fine with doing the upcasting but maybe it makes more sense to add it in as part of a more complicated builder as suggested by @emkornfield. 
   
   Like perhaps we have a class that takes all the column types (and in the future stats?) and it chooses the correct ordered byte representations and returns a ZOrder method for those specific input parameters.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800253357



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {
+            sourceByte++;
+            sourceBit = 7;
+          }
+        }
+      } while (columnsBinary[sourceColumn].length <= sourceByte && interleaveByte < interleavedSize);

Review comment:
       Does it make sense to have `interleaveBytes < interleavedSize` here? All that's going to happen is the first time through the loop, `sourceColumn` will be checked. But after that, if the interleave is finished then this will exit and the outer loop will exit. Then `sourceColumn` isn't used because the bytes are returned. So I think it would be equivalent to do this check above the do/while:
   
   ```
     if (interleaveBit == -1) {
       ...
     }
   
     if (interleaveByte < interleavedSize) {
       break;
     }
   
     do { ... } while (columnsBinary[sourceColumn].length <= sourceByte)
   ```
   
   You could also use an if block to avoid the `break` if that's your thing.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   Existing (this is essentially normal sorting by each element):
   | bits: l<sub>0</sub>, t<sub>1</sub>,  t<sub>0</sub> |*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   With padding:
   | bits: l<sub>1</sub>, t<sub>1</sub>, l<sub>0</sub>, t<sub>0</sub> |0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801084085



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       There's another simple option that I think solves an additional challenge...
   
   I'd propose that we cast everything to float so we don't have to worry about choosing a type and converting everything to it. That also gives us a more compact representation that is better for packing multiple columns. That's because floating point values are represented as scientific notation with base-2. The exponent encodes how many positions the fraction was shifted so that its first digit is a 1. Alternatively, you can think of it as encoding how many bits at the start of the number are 0, then storing a normalized fraction.
   
   That effectively compresses the most significant parts of a number into the highest bits. For float32, there's a sign bit, 8 bits of exponent, and 23 bits of fraction (mantissa).
   
   Converting all the values to floating point would solve the immediate problem because all numbers would use the same representation, giving what I think is the expected behavior (same result regardless of type). But it also means that we can fit more columns in 32 bits or 64 bits, which are going to be much faster for comparison once we pass them off to Spark because we could use native ints or longs rather than byte arrays.
   
   For example, if we took 4 integers and interleaved them, it would take 128 bits. But if we convert those to float first, then the most significant parts can fit in half that: 16 bits per value can carry the sign bit, the full 8-bit exponent, and 7 bits of the fraction. That's about 128 partitions of the number space per exponent value.
   
   If you don't want to lose so much of the value, then fewer columns do better. And we can always fall back to producing the full byte array if we want.
   
   What do you guys think?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801084085



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       There's another simple option that I think solves an additional challenge...
   
   I'd propose that we cast everything to float so we don't have to worry about choosing a type and converting everything to it. That also gives us a more compact representation that is better for packing multiple columns. That's because floating point values are represented as scientific notation with base-2. The exponent encodes how many positions the fraction was shifted so that its first digit is a 1. Alternatively, you can think of it as encoding how many bits at the start of the number are 0, then storing a normalized fraction.
   
   That effectively compresses the most significant parts of a number into the highest bits. For float32, there's a sign bit, 8 bits of exponent, and 23 bits of fraction (mantissa).
   
   Converting all the values to floating point would solve the immediate problem because all numbers would use the same representation, giving what I think is the expected behavior (same result regardless of type). But it also means that we can fit more columns in 32 bits or 64 bits, which are going to be much faster for comparison once we pass them off to Spark.
   
   For example, if we took 4 integers and interleaved them, it would take 128 bits. But if we convert those to float first, then the most significant parts can fit in half that: 16 bits per value can carry the sign bit, the full 8-bit exponent, and 7 bits of the fraction. That's about 128 partitions of the number space per exponent value.
   
   If you don't want to lose so much of the value, then fewer columns do better. And we can always fall back to producing the full byte array if we want.
   
   What do you guys think?




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802059958



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       > Should we only allow this when all the columns have the same length?
   
   `z_order(an_integer, a_bigint, 50_bytes_of_string)` -- each column has different byte length.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801078686



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);

Review comment:
       Implemented this but didn't allow for the "orAllocate" option. Caller needs to create the new allocation if they want to use 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794143536



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {
+      // The signed magnitude is positive set the first bit (reversing the sign so positives order after negatives)
+      floatBytes[0] = (byte) (floatBytes[0] | (1 << 7));
+    } else {
+      // The signed magnitude is negative so flip the first bit (reversing the sign so positives order after negatives)
+      // Then flip all remaining bits so numbers with greater negative magnitude come before those
+      // with less magnitude (reverse the order)
+      for (int i = 0; i < floatBytes.length; i++) {
+        floatBytes[i] = (byte) ~floatBytes[i];
+      }
+    }
+    return floatBytes;
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] orderUTF8LikeBytes(byte[] stringBytes, int size) {
+    if (stringBytes == null) {
+      return new byte[size];
+    }
+    return Arrays.copyOf(stringBytes, size);
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {

Review comment:
       I didn't mention it at first because it is an optimization but an alternative to this is a builder pattern that has specific methods for int/long/float/double/byte[] and a predertmined number of dimensions and a bit-width for each dimension.  then the addFloat/addInt methods could be called in the correct sequence.   I don't know if the JVM is smart enough to compile the ByteBuffer.of(...).put pattern to simple casts, but with the builder methods these could be avoided.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r795294600



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {

Review comment:
       We can just use the Hbase code here, the Bytes assumption is more a part of my UDF implementation on the Spark side. Otherwise I have to generate a udf of specific types based on the schema of column's being chosen which ... I think I may be able to do. I'll check it out




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r796129204



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {

Review comment:
       Changed all the methods to use java primitives so we should be ok here now, since the endianness is now Java's representation.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794169419



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {

Review comment:
       I should probably mark that I assume Java byte representations, good point




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r793249899



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {

Review comment:
       is it worth documenting assumed endianness and contract for nulls (always first in the sort 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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r793301219



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{
+        OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII,
+        IOIOIOIO, IOIOIOIO,
+        OIOIOIOI, OIOIOIOI,
+        OOOOIIII};
+    Assert.assertArrayEquals("Should combine mixed byte arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testIntOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      int aInt = random.nextInt();
+      int bInt = random.nextInt();
+      int intCompare = Integer.compare(aInt, bInt);
+      byte[] aBytes = ZOrderByteUtils.orderIntLikeBytes(bytesOf(aInt), 4);
+      byte[] bBytes = ZOrderByteUtils.orderIntLikeBytes(bytesOf(bInt), 4);
+      int byteCompare = UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes);
+
+      Assert.assertTrue(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aInt, bInt, intCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          (intCompare ^ byteCompare) >= 0);
+    }
+  }
+
+  @Test
+  public void testLongOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      long aLong = random.nextInt();
+      long bLong = random.nextInt();
+      int longCompare = Long.compare(aLong, bLong);
+      byte[] aBytes = ZOrderByteUtils.orderIntLikeBytes(bytesOf(aLong), 8);
+      byte[] bBytes = ZOrderByteUtils.orderIntLikeBytes(bytesOf(bLong), 8);
+      int byteCompare = UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes);
+
+      Assert.assertTrue(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aLong, bLong, longCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          (longCompare ^ byteCompare) >= 0);
+    }
+  }
+
+  @Test
+  public void testFloatOrdering() {
+    for (int i = 0; i < NUM_TESTS; i++) {
+      float aFloat = random.nextFloat();
+      float bFloat = random.nextFloat();
+      int floatCompare = Float.compare(aFloat, bFloat);
+      byte[] aBytes = ZOrderByteUtils.orderFloatLikeBytes(bytesOf(aFloat), 4);
+      byte[] bBytes = ZOrderByteUtils.orderFloatLikeBytes(bytesOf(bFloat), 4);
+      int byteCompare = UnsignedBytes.lexicographicalComparator().compare(aBytes, bBytes);
+
+      Assert.assertTrue(String.format(
+          "Ordering of ints should match ordering of bytes, %s ~ %s -> %s != %s ~ %s -> %s ",
+          aFloat, bFloat, floatCompare, Arrays.toString(aBytes), Arrays.toString(bBytes), byteCompare),
+          (floatCompare ^ byteCompare) >= 0);

Review comment:
       nit: This way of asserting the same comparison is a little bit opaque to me.  Comparing equality on [Integer.signum](https://docs.oracle.com/javase/7/docs/api/java/lang/Integer.html#signum(int)) would be clearer at least for me.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r793850604



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] orderIntLikeBytes(byte[] intBytes, int size) {
+    if (intBytes == null) {
+      return new byte[size];
+    }
+    intBytes[0] = (byte) (intBytes[0] ^ (1 << 7));
+    return intBytes;
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x \< y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] orderFloatLikeBytes(byte[] floatBytes, int size) {
+    if (floatBytes == null) {
+      return new byte[size];
+    }
+    if ((floatBytes[0] & (1 << 7)) == 0) {

Review comment:
       Yeah I can doc that, I think hopefully it will not come up very often. In the spec we already have a discussion on -/+ Nan #2891  I'm ok with different weird nans end up in random places at least for now




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r794169783



##########
File path: build.gradle
##########
@@ -222,6 +222,7 @@ project(':iceberg-core') {
     }
 
     testImplementation "org.xerial:sqlite-jdbc"
+    testImplementation "org.apache.commons:commons-lang3"

Review comment:
       Just for the test cases, it has an easy method for generating random strings of specific lengths. https://github.com/apache/iceberg/pull/3966/files#diff-2f797a408a34d59d8b0d0a0502cf96af670e693be34f18768579ba85b6fdb45dR216




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r802628105



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {

Review comment:
       > A. : All bytes that are available
   
   i think this is what we should do (Maybe with some max length for safety) because
   
   1. z-order on 1 column should be consistent with sort on that column
   2. adding a low-cardinality, short column (like boolean, or tinyint, or varchar(1) if we had it), should not reduce z-order sorting strength; so one short columns should not stop us from ordering longer columns




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800253357



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {
+            sourceByte++;
+            sourceBit = 7;
+          }
+        }
+      } while (columnsBinary[sourceColumn].length <= sourceByte && interleaveByte < interleavedSize);

Review comment:
       Does it make sense to have `interleaveBytes < interleavedSize` here? All that's going to happen is the first time through the loop, `sourceColumn` will be checked. But after that, if the interleave is finished then this will exit and the outer loop will exit. Then `sourceColumn` isn't used because the bytes are returned. So I think it would be equivalent to do this check above the do/while:
   
   ```java
     if (interleaveBit == -1) {
       ...
     }
   
     if (interleaveByte < interleavedSize) {
       break;
     }
   
     do { ... } while (columnsBinary[sourceColumn].length <= sourceByte)
   ```
   
   You could also use an if block to avoid the `break` if that's your thing.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801859352



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] longToOrderedBytes(long val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] shortToOrderedBytes(short val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, ByteBuffer)}
+   */
+  public static byte[] tinyintToOrderedBytes(byte val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Float.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)}
+   */
+  public static byte[] doubleToOrderedBytes(double val, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, Double.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length, ByteBuffer reuse) {
+    ByteBuffer bytes = ByteBuffers.reuse(reuse, length);
+    Arrays.fill(bytes.array(), 0, length, (byte) 0x00);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      // We may truncate mid-character
+      bytes.put(val.getBytes(StandardCharsets.UTF_8), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * For Testing interleave all available bytes
+   */
+  static byte[] interleaveBits(byte[][] columnsBinary) {
+    return interleaveBits(columnsBinary,
+        Arrays.stream(columnsBinary).mapToInt(column -> column.length).sum());

Review comment:
       I don't think so? Shouldn't we be allowing columns to contribute different amounts of bytes? 
   
   This is one of the difficulties for working with strings and other types since not only are we going to need more bytes for strings, but their most significant bytes may be rather late in the string.
   
   For example
   
   "www.iceberg.org/index"
   "www.iceberg.org/spark"
   
   Only have a differing byte at position 17.
   
   This is one of my reluctances to require similar contributions of byte sizes until we have a better "toOrderedBytes" function.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800756695



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    bytes.putInt(val ^ 0x80000000);
+    return bytes.array();
+  }
+
+  /**
+   * Signed longs are treated the same as the signed ints
+   */
+  public static byte[] longToOrderBytes(long val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    bytes.putLong(val ^ 0x8000000000000000L);
+    return bytes.array();
+  }
+
+  /**
+   * Signed shorts are treated the same as the signed ints
+   */
+  public static byte[] shortToOrderBytes(short val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Short.BYTES);
+    bytes.putShort((short) (val ^ (0x8000)));
+    return bytes.array();
+  }
+
+  /**
+   * Signed tiny ints are treated the same as the signed ints
+   */
+  public static byte[] tinyintToOrderedBytes(byte val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Byte.BYTES);
+    bytes.put((byte) (val ^ (0x80)));
+    return bytes.array();
+  }
+
+  /**
+   * IEEE 754 :
+   * “If two floating-point numbers in the same format are ordered (say, x {@literal <} y),
+   * they are ordered the same way when their bits are reinterpreted as sign-magnitude integers.”
+   *
+   * Which means floats can be treated as sign magnitude integers which can then be converted into lexicographically
+   * comparable bytes
+   */
+  public static byte[] floatToOrderedBytes(float val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);
+    int ival = Float.floatToIntBits(val);
+    ival ^= ((ival >> (Integer.SIZE - 1)) | Integer.MIN_VALUE);
+    bytes.putInt(ival);
+    return bytes.array();
+  }
+
+  /**
+   * Doubles are treated the same as floats
+   */
+  public static byte[] doubleToOrderedBytes(double val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Long.BYTES);
+    long lng = Double.doubleToLongBits(val);
+    lng ^= ((lng >> (Long.SIZE - 1)) | Long.MIN_VALUE);
+    bytes.putLong(lng);
+    return bytes.array();
+  }
+
+  /**
+   * Strings are lexicographically sortable BUT if different byte array lengths will
+   * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
+   * This implementation just uses a set size to for all output byte representations. Truncating longer strings
+   * and right padding 0 for shorter strings.
+   */
+  public static byte[] stringToOrderedBytes(String val, int length) {
+    ByteBuffer bytes = ByteBuffer.allocate(length);
+    if (val != null) {
+      int maxLength = Math.min(length, val.length());
+      bytes.put(val.getBytes(), 0, maxLength);
+    }
+    return bytes.array();
+  }
+
+  /**
+   * Interleave bits using a naive loop.
+   * @param columnsBinary an array of byte arrays, none of which are empty
+   * @return their bits interleaved
+   */
+  public static byte[] interleaveBits(byte[][] columnsBinary) {
+    int interleavedSize = Arrays.stream(columnsBinary).mapToInt(a -> a.length).sum();
+    byte[] interleavedBytes = new byte[interleavedSize];
+    int sourceBit = 7;
+    int sourceByte = 0;
+    int sourceColumn = 0;
+    int interleaveBit = 7;
+    int interleaveByte = 0;
+    while (interleaveByte < interleavedSize) {
+      // Take what we have, Get the source Bit of the source Byte, move it to the interleaveBit position
+      interleavedBytes[interleaveByte] =
+          (byte) (interleavedBytes[interleaveByte] |
+              (columnsBinary[sourceColumn][sourceByte] & 1 << sourceBit) >> sourceBit << interleaveBit);
+
+      --interleaveBit;
+      if (interleaveBit == -1) {
+        // Finished a byte in our interleave byte array start a new byte
+        interleaveByte++;
+        interleaveBit = 7;
+      }
+
+      // Find next column with a byte we can use
+      do {
+        ++sourceColumn;
+        if (sourceColumn == columnsBinary.length) {
+          sourceColumn = 0;
+          if (--sourceBit == -1) {
+            sourceByte++;
+            sourceBit = 7;
+          }
+        }
+      } while (columnsBinary[sourceColumn].length <= sourceByte && interleaveByte < interleavedSize);

Review comment:
       Yeah I'm not a big break person, but I think it complicated the code a bit since I wrote without them. Let me see what I can do to clean this up a bit




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800255152



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{OOOOOOOI, IIIIIIII, OOOOOOOO, OOOOIIII};
+    test[1] = new byte[]{OOOOOOOI, OOOOOOOO, IIIIIIII};
+    test[2] = new byte[]{OOOOOOOI};
+    test[3] = new byte[]{OOOOOOOI};
+    byte[] expected = new byte[]{
+        OOOOOOOO, OOOOOOOO, OOOOOOOO, OOOOIIII,
+        IOIOIOIO, IOIOIOIO,
+        OIOIOIOI, OIOIOIOI,
+        OOOOIIII};
+    Assert.assertArrayEquals("Should combine mixed byte arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testIntOrdering() {

Review comment:
       Good tests.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   Existing:
   ||*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   With padding:
   ||0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   ||*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   ||0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800711855



##########
File path: core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.iceberg.util;
+
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.iceberg.relocated.com.google.common.primitives.UnsignedBytes;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestZOrderByteUtil {
+  private static final byte IIIIIIII = (byte) 255;
+  private static final byte IOIOIOIO = (byte) 170;
+  private static final byte OIOIOIOI = (byte) 85;
+  private static final byte OOOOIIII = (byte) 15;
+  private static final byte OOOOOOOI = (byte) 1;
+  private static final byte OOOOOOOO = (byte) 0;
+
+  private static final int NUM_TESTS = 100000;
+
+  private final Random random = new Random(42);
+
+  private String bytesToString(byte[] bytes) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : bytes) {
+      result.append(String.format("%8s", Integer.toBinaryString(b & 0xFF)).replace(' ', '0'));
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns a non-0 length byte array
+   */
+  private byte[]  generateRandomBytes() {
+    int length = Math.abs(random.nextInt(100) + 1);
+    byte[] result = new byte[length];
+    random.nextBytes(result);
+    return result;
+  }
+
+  /**
+   * Test method to ensure correctness of byte interleaving code
+   */
+  private String interleaveStrings(String[] strings) {
+    StringBuilder result = new StringBuilder();
+    int totalLength = Arrays.stream(strings).mapToInt(String::length).sum();
+    int substringIndex = 0;
+    int characterIndex = 0;
+    while (characterIndex < totalLength) {
+      for (String str : strings) {
+        if (substringIndex < str.length()) {
+          result.append(str.charAt(substringIndex));
+          characterIndex++;
+        }
+      }
+      substringIndex++;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Compares the result of a string based interleaving algorithm implemented above
+   * versus the binary bit-shifting algorithm used in ZOrderByteUtils. Either both
+   * algorithms are identically wrong or are both identically correct.
+   */
+  @Test
+  public void testInterleaveRandomExamples() {
+    for (int test = 0; test < NUM_TESTS; test++) {
+      int numByteArrays = Math.abs(random.nextInt(6)) + 1;
+      byte[][] testBytes =  new byte[numByteArrays][];
+      String[] testStrings = new String[numByteArrays];
+      for (int byteIndex = 0;  byteIndex < numByteArrays; byteIndex++) {
+        testBytes[byteIndex] = generateRandomBytes();
+        testStrings[byteIndex] = bytesToString(testBytes[byteIndex]);
+      }
+      byte[] byteResult = ZOrderByteUtils.interleaveBits(testBytes);
+      String byteResultAsString = bytesToString(byteResult);
+
+      String stringResult = interleaveStrings(testStrings);
+
+      Assert.assertEquals("String interleave didn't match byte interleave", stringResult, byteResultAsString);
+    }
+  }
+
+  @Test
+  public void testInterleaveEmptyBits() {
+    byte[][] test = new byte[4][10];
+    byte[] expected = new byte[40];
+
+    Assert.assertArrayEquals("Should combine empty arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveFullBits() {
+    byte[][] test = new byte[4][];
+    test[0] = new byte[]{IIIIIIII, IIIIIIII};
+    test[1] = new byte[]{IIIIIIII};
+    test[2] = new byte[0];
+    test[3] = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII};
+    byte[] expected = new byte[]{IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII, IIIIIIII};
+
+    Assert.assertArrayEquals("Should combine full arrays",
+        expected, ZOrderByteUtils.interleaveBits(test));
+  }
+
+  @Test
+  public void testInterleaveMixedBits() {

Review comment:
       Thinking about this some more, I think the padding (casting to highest bit-width) is the right thing to do (below are tables showing 1 x 2 bit examples (I think what @rdblue  meant by useful vs not-useful can be seen by following).  The top and outer left most columns represent the input numbers
   
   Existing:
   | bits: n2_0, n1_1,  n1_0 |*0* | *1* | *2* | *3*
   |-|-|-|-|-|
   |0|0|1|2|3|
   |1|4|5|6|7|
   
   With padding:
   | n2_1, n1_1, n2_0, n1_0 |0 | 1 | 2 | 3
   |-|-|-|-|-|
   |0|0|1|4|5|
   |1 (padded as `0b01`)|2|3|6|7|
   
   




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r800766097



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.iceberg.util;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * Within Z-Ordering the byte representations of objects being compared must be ordered,
+ * this requires several types to be transformed when converted to bytes. The goal is to
+ * map object's whose byte representation are not lexicographically ordered into representations
+ * that are lexicographically ordered.
+ * Most of these techniques are derived from
+ * https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ *
+ * Some implementation is taken from
+ * https://github.com/apache/hbase/blob/master/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+ */
+public class ZOrderByteUtils {
+
+  private ZOrderByteUtils() {
+
+  }
+
+  /**
+   * Signed ints do not have their bytes in magnitude order because of the sign bit.
+   * To fix this, flip the sign bit so that all negatives are ordered before positives. This essentially
+   * shifts the 0 value so that we don't break our ordering when we cross the new 0 value.
+   */
+  public static byte[] intToOrderedBytes(int val) {
+    ByteBuffer bytes = ByteBuffer.allocate(Integer.BYTES);

Review comment:
       Actually I wonder if we should have reuseOrAllocate or just reuse and throw an error if it cannot. Basically give two versions of the orderedByteFunctions. One which attempts to reuse and throws an error if the Buffer cannot be reused and one that allocates a new buffer if no buffer is passed. That way the user knows their buffer is being re-used and the behavior isn't changing at runtime.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] findepi commented on a change in pull request #3966: Core: Adds Utility Class for Implementing ZOrdering

Posted by GitBox <gi...@apache.org>.
findepi commented on a change in pull request #3966:
URL: https://github.com/apache/iceberg/pull/3966#discussion_r801579852



##########
File path: core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java
##########
@@ -108,54 +108,70 @@ private ZOrderByteUtils() {
    * Strings are lexicographically sortable BUT if different byte array lengths will
    * ruin the Z-Ordering. (ZOrder requires that a given column contribute the same number of bytes every time).
    * This implementation just uses a set size to for all output byte representations. Truncating longer strings
-   * and right padding 0 for shorter strings.
+   * and right padding 0 for shorter strings. Requires UTF8 (or ASCII) encoding for ordering guarantees to hold.

Review comment:
       Please don't add a comment. Just replace `String.getBytes()` with `String.getBytes(java.nio.charset.StandardCharsets#UTF_8)`.
   
   Of course, i don't know the rules of the project, but otherwise i don't see why we would want to rely on JVM's default encoding. This is z-order, not sorting, but z-order should be consistent with sorting, and varchar value ordering is codepoint-based (in the absence of collation which doesn't exist in Iceberg world). The UTF-8 encoding produces bytes that, if treated as unsigned, lead to the same order as codepoint-based lexicographical ordering. 




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



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