szehon-ho commented on code in PR #3983:
URL: https://github.com/apache/iceberg/pull/3983#discussion_r853581777


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderStrategy.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.spark.actions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.SortOrderUtil;
+import org.apache.iceberg.util.ZOrderByteUtils;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.distributions.Distributions;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkZOrderStrategy extends SparkSortStrategy {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkZOrderStrategy.class);
+
+  private static final String Z_COLUMN = "ICEZVALUE";
+  private static final Schema Z_SCHEMA = new Schema(NestedField.required(0, 
Z_COLUMN, Types.BinaryType.get()));
+  private static final org.apache.iceberg.SortOrder Z_SORT_ORDER = 
org.apache.iceberg.SortOrder.builderFor(Z_SCHEMA)
+      .sortBy(Z_COLUMN, SortDirection.ASC, NullOrder.NULLS_LAST)
+      .build();
+
+  /**
+   * Controls the amount of bytes interleaved in the ZOrder Algorithm. Default 
is all bytes being interleaved.

Review Comment:
   For learning, what is the effect if we do not use all bytes?



##########
core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.CharBuffer;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * 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. Bytes produced should be compared 
lexicographically as
+ * unsigned bytes, big-endian.
+ * <p>
+ * All types except for String are stored within an 8 Byte Buffer
+ * <p>
+ * Most of these techniques are derived from
+ * 
https://aws.amazon.com/blogs/database/z-order-indexing-for-multifaceted-queries-in-amazon-dynamodb-part-2/
+ * <p>
+ * 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 {
+
+  public static final int PRIMITIVE_BUFFER_SIZE = 8;
+
+  private ZOrderByteUtils() {
+

Review Comment:
   Nit: extra space (other Util has 1 space)



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.actions;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import org.apache.iceberg.util.ZOrderByteUtils;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.expressions.UserDefinedFunction;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.BinaryType;
+import org.apache.spark.sql.types.BooleanType;
+import org.apache.spark.sql.types.ByteType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DateType;
+import org.apache.spark.sql.types.DoubleType;
+import org.apache.spark.sql.types.FloatType;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.LongType;
+import org.apache.spark.sql.types.ShortType;
+import org.apache.spark.sql.types.StringType;
+import org.apache.spark.sql.types.TimestampType;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+class SparkZOrderUDF implements Serializable {
+  private static final byte[] PRIMITIVE_EMPTY = new 
byte[ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE];
+
+  /**
+   * Every Spark task runs iteratively on a rows in a single thread so 
ThreadLocal should protect from
+   * concurrent access to any of these structures.
+   */
+  private transient ThreadLocal<ByteBuffer> outputBuffer;
+  private transient ThreadLocal<byte[][]> inputHolder;
+  private transient ThreadLocal<ByteBuffer[]> inputBuffers;
+  private transient ThreadLocal<CharsetEncoder> encoder;
+
+  private final int numCols;
+
+  private int inputCol = 0;
+  private int totalOutputBytes = 0;
+  private final int varTypeSize;
+  private final int maxOutputSize;
+
+  SparkZOrderUDF(int numCols, int varTypeSize, int maxOutputSize) {
+    this.numCols = numCols;
+    this.varTypeSize = varTypeSize;
+    this.maxOutputSize = maxOutputSize;
+  }
+
+  private void readObject(ObjectInputStream in) throws IOException, 
ClassNotFoundException {
+    in.defaultReadObject();
+    inputBuffers = ThreadLocal.withInitial(() -> new ByteBuffer[numCols]);
+    inputHolder = ThreadLocal.withInitial(() -> new byte[numCols][]);
+    outputBuffer = ThreadLocal.withInitial(() -> 
ByteBuffer.allocate(totalOutputBytes));
+    encoder = ThreadLocal.withInitial(() -> 
StandardCharsets.UTF_8.newEncoder());
+  }
+
+  private ByteBuffer inputBuffer(int position, int size) {
+    ByteBuffer buffer = inputBuffers.get()[position];
+    if (buffer == null) {
+      buffer = ByteBuffer.allocate(size);
+      inputBuffers.get()[position] = buffer;
+    }
+    return buffer;
+  }
+
+  byte[] interleaveBits(Seq<byte[]> scalaBinary) {
+    byte[][] columnsBinary = 
JavaConverters.seqAsJavaList(scalaBinary).toArray(inputHolder.get());
+    return ZOrderByteUtils.interleaveBits(columnsBinary, totalOutputBytes, 
outputBuffer.get());
+  }
+
+  private UserDefinedFunction tinyToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Byte value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.tinyintToOrderedBytes(value, 
inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("TINY_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction shortToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Short value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.shortToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("SHORT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction intToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Integer value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.intToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("INT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction longToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Long value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.longToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("LONG_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction floatToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Float value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.floatToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("FLOAT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction doubleToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Double value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.doubleToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("DOUBLE_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction stringToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((String value) ->
+        ZOrderByteUtils.stringToOrderedBytes(
+            value,
+            varTypeSize,
+            inputBuffer(position, varTypeSize),
+            encoder.get()).array(), DataTypes.BinaryType)
+          .withName("STRING-LEXICAL-BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(varTypeSize);
+
+    return udf;
+  }
+
+  private UserDefinedFunction bytesTruncateUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((byte[] value) ->
+        ZOrderByteUtils.byteTruncateOrFill(value, varTypeSize, 
inputBuffer(position, varTypeSize)).array(),
+            DataTypes.BinaryType)
+        .withName("BYTE-TRUNCATE");
+
+    this.inputCol++;
+    increaseOutputSize(varTypeSize);
+
+    return udf;
+  }
+
+  private UserDefinedFunction booleanToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Boolean value) -> {
+      ByteBuffer buffer = inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+      buffer.put(0, (byte) (value ? -127 : 0));
+      return buffer.array();
+    }, DataTypes.BinaryType)
+        .withName("BOOLEAN-LEXICAL-BYTES");

Review Comment:
   Nit: looks a bit weird, maybe put on previous line like others



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderStrategy.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.spark.actions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.RewriteStrategy;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.SortOrderUtil;
+import org.apache.iceberg.util.ZOrderByteUtils;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.distributions.Distributions;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkZOrderStrategy extends SparkSortStrategy {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkZOrderStrategy.class);
+
+  private static final String Z_COLUMN = "ICEZVALUE";
+  private static final Schema Z_SCHEMA = new Schema(NestedField.required(0, 
Z_COLUMN, Types.BinaryType.get()));
+  private static final org.apache.iceberg.SortOrder Z_SORT_ORDER = 
org.apache.iceberg.SortOrder.builderFor(Z_SCHEMA)
+      .sortBy(Z_COLUMN, SortDirection.ASC, NullOrder.NULLS_LAST)
+      .build();
+
+  /**
+   * Controls the amount of bytes interleaved in the ZOrder Algorithm. Default 
is all bytes being interleaved.
+   */
+  private static final String MAX_OUTPUT_SIZE_KEY = "max-output-size";
+  private static final int DEFAULT_MAX_OUTPUT_SIZE = Integer.MAX_VALUE;
+
+  /**
+  * Controls the number of bytes considered from an input column of a type 
with variable length (String, Binary).
+  * Default is to use the same size as primitives {@link 
ZOrderByteUtils#PRIMITIVE_BUFFER_SIZE}
+   */
+  private static final String VAR_LENGTH_CONTRIBUTION_KEY = 
"var-length-contribution";
+  private static final int DEFAULT_VAR_LENGTH_CONTRIBUTION = 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE;
+
+  private final List<String> zOrderColNames;
+
+  private int maxOutputSize;
+  private int varLengthContribution;
+
+  @Override
+  public Set<String> validOptions() {
+    return ImmutableSet.<String>builder()
+        .addAll(super.validOptions())
+        .add(VAR_LENGTH_CONTRIBUTION_KEY)
+        .add(MAX_OUTPUT_SIZE_KEY)
+        .build();
+  }
+
+  @Override
+  public RewriteStrategy options(Map<String, String> options) {
+    super.options(options);
+
+    varLengthContribution = PropertyUtil.propertyAsInt(options, 
VAR_LENGTH_CONTRIBUTION_KEY,
+        DEFAULT_VAR_LENGTH_CONTRIBUTION);
+    Preconditions.checkArgument(varLengthContribution > 0,
+        "Cannot use less than 1 byte for variable length types with zOrder, %s 
was set to %s",
+        VAR_LENGTH_CONTRIBUTION_KEY, varLengthContribution);
+
+    maxOutputSize = PropertyUtil.propertyAsInt(options, MAX_OUTPUT_SIZE_KEY, 
DEFAULT_MAX_OUTPUT_SIZE);
+    Preconditions.checkArgument(maxOutputSize > 0,
+        "Cannot have the interleaved ZOrder value use less than 1 byte, %s was 
set to %s",
+        MAX_OUTPUT_SIZE_KEY, maxOutputSize);
+
+    return this;
+  }
+
+  public SparkZOrderStrategy(Table table, SparkSession spark, List<String> 
zOrderColNames) {
+    super(table, spark);
+
+    Preconditions.checkArgument(zOrderColNames != null && 
!zOrderColNames.isEmpty(),
+        "Cannot ZOrder when no columns are specified");
+
+    Stream<String> identityPartitionColumns = table.spec().fields().stream()
+        .filter(f -> f.transform().isIdentity())
+        .map(PartitionField::name);
+    List<String> partZOrderCols = identityPartitionColumns
+        .filter(zOrderColNames::contains)
+        .collect(Collectors.toList());
+
+    if (!partZOrderCols.isEmpty()) {
+      LOG.warn("Cannot ZOrder on an Identity partition column as these values 
are constant within a partition " +
+                      "they will be removed from the ZOrder expression: {}", 
partZOrderCols);

Review Comment:
   nit:  "they" is a bit abrupt for the middle of the sentence, how about "and"



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.actions;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.StandardCharsets;
+import org.apache.iceberg.util.ZOrderByteUtils;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.expressions.UserDefinedFunction;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.BinaryType;
+import org.apache.spark.sql.types.BooleanType;
+import org.apache.spark.sql.types.ByteType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DateType;
+import org.apache.spark.sql.types.DoubleType;
+import org.apache.spark.sql.types.FloatType;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.LongType;
+import org.apache.spark.sql.types.ShortType;
+import org.apache.spark.sql.types.StringType;
+import org.apache.spark.sql.types.TimestampType;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+class SparkZOrderUDF implements Serializable {
+  private static final byte[] PRIMITIVE_EMPTY = new 
byte[ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE];
+
+  /**
+   * Every Spark task runs iteratively on a rows in a single thread so 
ThreadLocal should protect from
+   * concurrent access to any of these structures.
+   */
+  private transient ThreadLocal<ByteBuffer> outputBuffer;
+  private transient ThreadLocal<byte[][]> inputHolder;
+  private transient ThreadLocal<ByteBuffer[]> inputBuffers;
+  private transient ThreadLocal<CharsetEncoder> encoder;
+
+  private final int numCols;
+
+  private int inputCol = 0;
+  private int totalOutputBytes = 0;
+  private final int varTypeSize;
+  private final int maxOutputSize;
+
+  SparkZOrderUDF(int numCols, int varTypeSize, int maxOutputSize) {
+    this.numCols = numCols;
+    this.varTypeSize = varTypeSize;
+    this.maxOutputSize = maxOutputSize;
+  }
+
+  private void readObject(ObjectInputStream in) throws IOException, 
ClassNotFoundException {
+    in.defaultReadObject();
+    inputBuffers = ThreadLocal.withInitial(() -> new ByteBuffer[numCols]);
+    inputHolder = ThreadLocal.withInitial(() -> new byte[numCols][]);
+    outputBuffer = ThreadLocal.withInitial(() -> 
ByteBuffer.allocate(totalOutputBytes));
+    encoder = ThreadLocal.withInitial(() -> 
StandardCharsets.UTF_8.newEncoder());
+  }
+
+  private ByteBuffer inputBuffer(int position, int size) {
+    ByteBuffer buffer = inputBuffers.get()[position];
+    if (buffer == null) {
+      buffer = ByteBuffer.allocate(size);
+      inputBuffers.get()[position] = buffer;
+    }
+    return buffer;
+  }
+
+  byte[] interleaveBits(Seq<byte[]> scalaBinary) {
+    byte[][] columnsBinary = 
JavaConverters.seqAsJavaList(scalaBinary).toArray(inputHolder.get());
+    return ZOrderByteUtils.interleaveBits(columnsBinary, totalOutputBytes, 
outputBuffer.get());
+  }
+
+  private UserDefinedFunction tinyToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Byte value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.tinyintToOrderedBytes(value, 
inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("TINY_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction shortToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Short value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.shortToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("SHORT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction intToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Integer value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.intToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("INT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction longToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Long value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.longToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("LONG_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction floatToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Float value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.floatToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("FLOAT_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction doubleToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Double value) -> {
+      if (value == null) {
+        return PRIMITIVE_EMPTY;
+      }
+      return ZOrderByteUtils.doubleToOrderedBytes(value, inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE))
+          .array();
+    }, DataTypes.BinaryType).withName("DOUBLE_ORDERED_BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+
+    return udf;
+  }
+
+  private UserDefinedFunction stringToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((String value) ->
+        ZOrderByteUtils.stringToOrderedBytes(
+            value,
+            varTypeSize,
+            inputBuffer(position, varTypeSize),
+            encoder.get()).array(), DataTypes.BinaryType)
+          .withName("STRING-LEXICAL-BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(varTypeSize);
+
+    return udf;
+  }
+
+  private UserDefinedFunction bytesTruncateUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((byte[] value) ->
+        ZOrderByteUtils.byteTruncateOrFill(value, varTypeSize, 
inputBuffer(position, varTypeSize)).array(),
+            DataTypes.BinaryType)
+        .withName("BYTE-TRUNCATE");
+
+    this.inputCol++;
+    increaseOutputSize(varTypeSize);
+
+    return udf;
+  }
+
+  private UserDefinedFunction booleanToOrderedBytesUDF() {
+    int position = inputCol;
+    UserDefinedFunction udf = functions.udf((Boolean value) -> {
+      ByteBuffer buffer = inputBuffer(position, 
ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+      buffer.put(0, (byte) (value ? -127 : 0));
+      return buffer.array();
+    }, DataTypes.BinaryType)
+        .withName("BOOLEAN-LEXICAL-BYTES");
+
+    this.inputCol++;
+    increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE);
+    return udf;
+  }
+
+  private final UserDefinedFunction interleaveUDF =
+      functions.udf((Seq<byte[]> arrayBinary) -> interleaveBits(arrayBinary), 
DataTypes.BinaryType)
+          .withName("INTERLEAVE_BYTES");
+
+  Column interleaveBytes(Column arrayBinary) {
+    return interleaveUDF.apply(arrayBinary);
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  Column sortedLexicographically(Column column, DataType type) {
+    if (type instanceof ByteType) {
+      return tinyToOrderedBytesUDF().apply(column);
+    } else if (type instanceof ShortType) {
+      return shortToOrderedBytesUDF().apply(column);
+    } else if (type instanceof IntegerType) {
+      return intToOrderedBytesUDF().apply(column);
+    } else if (type instanceof LongType) {
+      return longToOrderedBytesUDF().apply(column);
+    } else if (type instanceof FloatType) {
+      return floatToOrderedBytesUDF().apply(column);
+    } else if (type instanceof DoubleType) {
+      return doubleToOrderedBytesUDF().apply(column);
+    } else if (type instanceof StringType) {
+      return stringToOrderedBytesUDF().apply(column);
+    } else if (type instanceof BinaryType) {
+      return bytesTruncateUDF().apply(column);
+    } else if (type instanceof BooleanType) {
+      return booleanToOrderedBytesUDF().apply(column);
+    } else if (type instanceof TimestampType) {
+      return longToOrderedBytesUDF().apply(column.cast(DataTypes.LongType));
+    } else if (type instanceof DateType) {
+      return longToOrderedBytesUDF().apply(column.cast(DataTypes.LongType));
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Cannot use column %s of type %s in ZOrdering, the 
type is unsupported", column, type));
+    }
+  }
+
+  private void increaseOutputSize(int bytes) {
+    totalOutputBytes += bytes;
+    if (totalOutputBytes > maxOutputSize) {

Review Comment:
   Optional:  totalOutputBytes = Math.max(totalOutputBytes, maxOutputSize) can 
fit on one line



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to