gemini-code-assist[bot] commented on code in PR #38406:
URL: https://github.com/apache/beam/pull/38406#discussion_r3202710866


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sorter.BufferedExternalSorter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.SortField;
+import org.apache.iceberg.SortOrder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.ReadableInstant;
+
+/**
+ * A utility class to sort Beam {@link Row}s based on an Iceberg {@link 
SortOrder}. Leverages {@link
+ * BufferedExternalSorter} to spill to local disk when elements exceed memory 
limit.
+ */
+class IcebergRowSorter implements Serializable {
+
+  public static Iterable<Row> sortRows(
+      Iterable<Row> rows,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema) {
+
+    if (sortOrder == null || !sortOrder.isSorted()) {
+      return rows;
+    }
+
+    BufferedExternalSorter.Options sorterOptions = 
BufferedExternalSorter.options();
+    BufferedExternalSorter sorter = 
BufferedExternalSorter.create(sorterOptions);
+    RowCoder rowCoder = RowCoder.of(beamSchema);
+
+    try {
+      for (Row row : rows) {
+        byte[] keyBytes = encodeSortKey(row, sortOrder, icebergSchema, 
beamSchema);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        rowCoder.encode(row, baos);
+        byte[] valBytes = baos.toByteArray();
+        sorter.add(KV.of(keyBytes, valBytes));
+      }
+
+      Iterable<KV<byte[], byte[]>> sortedKVs = sorter.sort();
+      return new Iterable<Row>() {
+        @Override
+        public Iterator<Row> iterator() {
+          final Iterator<KV<byte[], byte[]>> it = sortedKVs.iterator();
+          return new Iterator<Row>() {
+            @Override
+            public boolean hasNext() {
+              return it.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              KV<byte[], byte[]> next = it.next();
+              try {
+                ByteArrayInputStream bais = new 
ByteArrayInputStream(next.getValue());
+                return rowCoder.decode(bais);
+              } catch (IOException e) {
+                throw new RuntimeException("Failed to decode Row during 
sorting", e);
+              }
+            }
+          };
+        }
+      };
+
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to sort rows with external sorter", 
e);
+    }
+  }
+
+  @SuppressWarnings("nullness")
+  public static byte[] encodeSortKey(
+      Row row,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema)
+      throws IOException {
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    for (SortField field : sortOrder.fields()) {
+      String colName = icebergSchema.findColumnName(field.sourceId());
+      Object val = row.getValue(colName);
+
+      if (!field.transform().isIdentity()) {
+        Object icebergVal =
+            IcebergUtils.beamRowToIcebergRecord(icebergSchema, 
row).getField(colName);
+        if (icebergVal != null) {
+          val = field.transform().apply(icebergVal);
+        } else {
+          val = null;
+        }
+      }
+
+      boolean isNull = (val == null);
+      boolean isDesc = (field.direction() == SortDirection.DESC);
+      boolean nullsFirst = (field.nullOrder() == NullOrder.NULLS_FIRST);
+
+      // Determine correct header prefix to fulfill the NullOrder contracts
+      byte prefixByte;
+      if (isNull) {
+        if (isDesc) {
+          // Descending: High byte keys sort first.
+          // If Nulls First -> Null gets highest byte (0xFF)
+          // If Nulls Last -> Null gets lowest byte (0x00)
+          prefixByte = nullsFirst ? (byte) 0xFF : (byte) 0x00;
+        } else {
+          // Ascending: Low byte keys sort first.
+          // If Nulls First -> Null gets lowest byte (0x00)
+          // If Nulls Last -> Null gets highest byte (0xFF)
+          prefixByte = nullsFirst ? (byte) 0x00 : (byte) 0xFF;
+        }
+      } else {
+        if (isDesc) {
+          // If non-null and Descending, use a neutral value that sits 
opposite to the null byte
+          prefixByte = nullsFirst ? (byte) 0xFE : (byte) 0x01;
+        } else {
+          prefixByte = nullsFirst ? (byte) 0x01 : (byte) 0x00;
+        }
+      }

Review Comment:
   ![critical](https://www.gstatic.com/codereviewagent/critical.svg)
   
   The logic for `prefixByte` when `isDesc` is true is incorrect. 
`BufferedExternalSorter` performs an ascending lexicographical comparison of 
the byte keys. To achieve `NULLS_FIRST`, the null prefix must always be smaller 
than the non-null prefix, regardless of whether the column values are inverted 
for descending order. Currently, for `isDesc=true`, `NULLS_FIRST` uses `0xFF` 
for null and `0xFE` for non-null, which will cause nulls to sort **last** in 
the resulting iterable.
   
   Additionally, the prefix logic can be simplified as it does not depend on 
the sort direction; the direction is already handled by inverting the value 
bytes.
   
   ```java
         // Determine correct header prefix to fulfill the NullOrder contracts
         byte prefixByte;
         if (isNull) {
           prefixByte = nullsFirst ? (byte) 0x00 : (byte) 0xFF;
         } else {
           prefixByte = nullsFirst ? (byte) 0x01 : (byte) 0x00;
         }
   ```



##########
sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorterTest.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.types.Types;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class IcebergRowSorterTest {
+
+  private static final Schema BEAM_SCHEMA =
+      Schema.builder()
+          .addInt32Field("id")
+          .addNullableField("name", Schema.FieldType.STRING)
+          .addNullableField("value", Schema.FieldType.DOUBLE)
+          .addNullableField("active", Schema.FieldType.BOOLEAN)
+          .build();
+
+  private static final org.apache.iceberg.Schema ICEBERG_SCHEMA =
+      new org.apache.iceberg.Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "name", Types.StringType.get()),
+          Types.NestedField.optional(3, "value", Types.DoubleType.get()),
+          Types.NestedField.optional(4, "active", Types.BooleanType.get()));
+
+  private static final Comparator<byte[]> BYTE_ARR_COMPARATOR =
+      
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedBytes
+          .lexicographicalComparator();
+
+  @Test
+  public void testStringKeyEncodingOrder() throws Exception {
+    SortOrder sortOrder = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("name").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "apple", 1.5, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(2, "banana", 2.0, 
true).build();
+    Row r3 = Row.withSchema(BEAM_SCHEMA).addValues(3, "apricot", 3.0, 
false).build();
+
+    byte[] k1 = IcebergRowSorter.encodeSortKey(r1, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k2 = IcebergRowSorter.encodeSortKey(r2, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k3 = IcebergRowSorter.encodeSortKey(r3, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k2) < 0); // apple < banana
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k3) < 0); // apple < apricot
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k3, k2) < 0); // apricot < banana
+  }
+
+  @Test
+  public void testStringCollisionProofing() throws Exception {
+    // Tests that secondary columns don't bleed into primary columns.
+    // Row 1: Primary="abc", Secondary="def"
+    // Row 2: Primary="abcdef", Secondary=null
+    // In raw byte concatenation, both could equal "abcdef\0" if delimiters or 
escaping fail.
+    SortOrder sortOrder = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("name").asc("value").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "abc", 1.0, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(2, "abcdef", null, 
true).build();
+
+    byte[] k1 = IcebergRowSorter.encodeSortKey(r1, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k2 = IcebergRowSorter.encodeSortKey(r2, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+
+    // "abc" must sort lexicographically before "abcdef"
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k2) < 0);
+  }
+
+  @Test
+  public void testDescInversion() throws Exception {
+    SortOrder sortOrderAsc = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("id").build();
+    SortOrder sortOrderDesc = 
SortOrder.builderFor(ICEBERG_SCHEMA).desc("id").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(10, "test", 1.5, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(20, "test", 2.0, 
true).build();
+
+    byte[] k1Asc = IcebergRowSorter.encodeSortKey(r1, sortOrderAsc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+    byte[] k2Asc = IcebergRowSorter.encodeSortKey(r2, sortOrderAsc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+
+    byte[] k1Desc = IcebergRowSorter.encodeSortKey(r1, sortOrderDesc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+    byte[] k2Desc = IcebergRowSorter.encodeSortKey(r2, sortOrderDesc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+
+    // Ascending: 10 < 20
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1Asc, k2Asc) < 0);
+
+    // Descending: 10 > 20 (inverted bytes)
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1Desc, k2Desc) > 0);
+  }
+
+  @Test
+  public void testNullOrderingMatrix() throws Exception {
+    Row rNonNull = Row.withSchema(BEAM_SCHEMA).addValues(1, "apple", 1.5, 
true).build();
+    Row rNull = Row.withSchema(BEAM_SCHEMA).addValues(2, null, 2.0, 
true).build();
+
+    // 1. ASC, NULLS_FIRST
+    SortOrder ascFirst =
+        SortOrder.builderFor(ICEBERG_SCHEMA).asc("name", 
NullOrder.NULLS_FIRST).build();
+    byte[] kNonNullAscFirst =
+        IcebergRowSorter.encodeSortKey(rNonNull, ascFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullAscFirst =
+        IcebergRowSorter.encodeSortKey(rNull, ascFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "ASC NULLS_FIRST failed: null should sort before non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullAscFirst, kNonNullAscFirst) < 0);
+
+    // 2. ASC, NULLS_LAST
+    SortOrder ascLast =
+        SortOrder.builderFor(ICEBERG_SCHEMA).asc("name", 
NullOrder.NULLS_LAST).build();
+    byte[] kNonNullAscLast =
+        IcebergRowSorter.encodeSortKey(rNonNull, ascLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullAscLast =
+        IcebergRowSorter.encodeSortKey(rNull, ascLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "ASC NULLS_LAST failed: null should sort after non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullAscLast, kNonNullAscLast) > 0);
+
+    // 3. DESC, NULLS_FIRST
+    SortOrder descFirst =
+        SortOrder.builderFor(ICEBERG_SCHEMA).desc("name", 
NullOrder.NULLS_FIRST).build();
+    byte[] kNonNullDescFirst =
+        IcebergRowSorter.encodeSortKey(rNonNull, descFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullDescFirst =
+        IcebergRowSorter.encodeSortKey(rNull, descFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "DESC NULLS_FIRST failed: null should sort before non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullDescFirst, kNonNullDescFirst) > 0);
+
+    // 4. DESC, NULLS_LAST
+    SortOrder descLast =
+        SortOrder.builderFor(ICEBERG_SCHEMA).desc("name", 
NullOrder.NULLS_LAST).build();
+    byte[] kNonNullDescLast =
+        IcebergRowSorter.encodeSortKey(rNonNull, descLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullDescLast =
+        IcebergRowSorter.encodeSortKey(rNull, descLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "DESC NULLS_LAST failed: null should sort after non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullDescLast, kNonNullDescLast) < 0);
+  }

Review Comment:
   ![critical](https://www.gstatic.com/codereviewagent/critical.svg)
   
   This assertion is incorrect. For `NULLS_LAST`, the null key should be 
**larger** than the non-null key so that it appears last in an ascending sort. 
The current test asserts `< 0`, which validates that nulls appear first.
   
   ```java
       assertTrue(
           "DESC NULLS_LAST failed: null should sort after non-null",
           BYTE_ARR_COMPARATOR.compare(kNullDescLast, kNonNullDescLast) > 0);
   ```



##########
sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorterTest.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.types.Types;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class IcebergRowSorterTest {
+
+  private static final Schema BEAM_SCHEMA =
+      Schema.builder()
+          .addInt32Field("id")
+          .addNullableField("name", Schema.FieldType.STRING)
+          .addNullableField("value", Schema.FieldType.DOUBLE)
+          .addNullableField("active", Schema.FieldType.BOOLEAN)
+          .build();
+
+  private static final org.apache.iceberg.Schema ICEBERG_SCHEMA =
+      new org.apache.iceberg.Schema(
+          Types.NestedField.required(1, "id", Types.IntegerType.get()),
+          Types.NestedField.optional(2, "name", Types.StringType.get()),
+          Types.NestedField.optional(3, "value", Types.DoubleType.get()),
+          Types.NestedField.optional(4, "active", Types.BooleanType.get()));
+
+  private static final Comparator<byte[]> BYTE_ARR_COMPARATOR =
+      
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedBytes
+          .lexicographicalComparator();
+
+  @Test
+  public void testStringKeyEncodingOrder() throws Exception {
+    SortOrder sortOrder = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("name").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "apple", 1.5, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(2, "banana", 2.0, 
true).build();
+    Row r3 = Row.withSchema(BEAM_SCHEMA).addValues(3, "apricot", 3.0, 
false).build();
+
+    byte[] k1 = IcebergRowSorter.encodeSortKey(r1, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k2 = IcebergRowSorter.encodeSortKey(r2, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k3 = IcebergRowSorter.encodeSortKey(r3, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k2) < 0); // apple < banana
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k3) < 0); // apple < apricot
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k3, k2) < 0); // apricot < banana
+  }
+
+  @Test
+  public void testStringCollisionProofing() throws Exception {
+    // Tests that secondary columns don't bleed into primary columns.
+    // Row 1: Primary="abc", Secondary="def"
+    // Row 2: Primary="abcdef", Secondary=null
+    // In raw byte concatenation, both could equal "abcdef\0" if delimiters or 
escaping fail.
+    SortOrder sortOrder = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("name").asc("value").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "abc", 1.0, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(2, "abcdef", null, 
true).build();
+
+    byte[] k1 = IcebergRowSorter.encodeSortKey(r1, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] k2 = IcebergRowSorter.encodeSortKey(r2, sortOrder, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+
+    // "abc" must sort lexicographically before "abcdef"
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1, k2) < 0);
+  }
+
+  @Test
+  public void testDescInversion() throws Exception {
+    SortOrder sortOrderAsc = 
SortOrder.builderFor(ICEBERG_SCHEMA).asc("id").build();
+    SortOrder sortOrderDesc = 
SortOrder.builderFor(ICEBERG_SCHEMA).desc("id").build();
+
+    Row r1 = Row.withSchema(BEAM_SCHEMA).addValues(10, "test", 1.5, 
true).build();
+    Row r2 = Row.withSchema(BEAM_SCHEMA).addValues(20, "test", 2.0, 
true).build();
+
+    byte[] k1Asc = IcebergRowSorter.encodeSortKey(r1, sortOrderAsc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+    byte[] k2Asc = IcebergRowSorter.encodeSortKey(r2, sortOrderAsc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+
+    byte[] k1Desc = IcebergRowSorter.encodeSortKey(r1, sortOrderDesc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+    byte[] k2Desc = IcebergRowSorter.encodeSortKey(r2, sortOrderDesc, 
ICEBERG_SCHEMA, BEAM_SCHEMA);
+
+    // Ascending: 10 < 20
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1Asc, k2Asc) < 0);
+
+    // Descending: 10 > 20 (inverted bytes)
+    assertTrue(BYTE_ARR_COMPARATOR.compare(k1Desc, k2Desc) > 0);
+  }
+
+  @Test
+  public void testNullOrderingMatrix() throws Exception {
+    Row rNonNull = Row.withSchema(BEAM_SCHEMA).addValues(1, "apple", 1.5, 
true).build();
+    Row rNull = Row.withSchema(BEAM_SCHEMA).addValues(2, null, 2.0, 
true).build();
+
+    // 1. ASC, NULLS_FIRST
+    SortOrder ascFirst =
+        SortOrder.builderFor(ICEBERG_SCHEMA).asc("name", 
NullOrder.NULLS_FIRST).build();
+    byte[] kNonNullAscFirst =
+        IcebergRowSorter.encodeSortKey(rNonNull, ascFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullAscFirst =
+        IcebergRowSorter.encodeSortKey(rNull, ascFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "ASC NULLS_FIRST failed: null should sort before non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullAscFirst, kNonNullAscFirst) < 0);
+
+    // 2. ASC, NULLS_LAST
+    SortOrder ascLast =
+        SortOrder.builderFor(ICEBERG_SCHEMA).asc("name", 
NullOrder.NULLS_LAST).build();
+    byte[] kNonNullAscLast =
+        IcebergRowSorter.encodeSortKey(rNonNull, ascLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullAscLast =
+        IcebergRowSorter.encodeSortKey(rNull, ascLast, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "ASC NULLS_LAST failed: null should sort after non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullAscLast, kNonNullAscLast) > 0);
+
+    // 3. DESC, NULLS_FIRST
+    SortOrder descFirst =
+        SortOrder.builderFor(ICEBERG_SCHEMA).desc("name", 
NullOrder.NULLS_FIRST).build();
+    byte[] kNonNullDescFirst =
+        IcebergRowSorter.encodeSortKey(rNonNull, descFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    byte[] kNullDescFirst =
+        IcebergRowSorter.encodeSortKey(rNull, descFirst, ICEBERG_SCHEMA, 
BEAM_SCHEMA);
+    assertTrue(
+        "DESC NULLS_FIRST failed: null should sort before non-null",
+        BYTE_ARR_COMPARATOR.compare(kNullDescFirst, kNonNullDescFirst) > 0);
+

Review Comment:
   ![critical](https://www.gstatic.com/codereviewagent/critical.svg)
   
   This assertion is incorrect. For `NULLS_FIRST`, the null key should be 
**smaller** than the non-null key so that it appears first in an ascending 
sort. The current test asserts `> 0`, which validates that nulls appear last. 
This matches the bug in the implementation.
   
   ```java
       assertTrue(
           "DESC NULLS_FIRST failed: null should sort before non-null",
           BYTE_ARR_COMPARATOR.compare(kNullDescFirst, kNonNullDescFirst) < 0);
   ```



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sorter.BufferedExternalSorter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.SortField;
+import org.apache.iceberg.SortOrder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.ReadableInstant;
+
+/**
+ * A utility class to sort Beam {@link Row}s based on an Iceberg {@link 
SortOrder}. Leverages {@link
+ * BufferedExternalSorter} to spill to local disk when elements exceed memory 
limit.
+ */
+class IcebergRowSorter implements Serializable {
+
+  public static Iterable<Row> sortRows(
+      Iterable<Row> rows,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema) {
+
+    if (sortOrder == null || !sortOrder.isSorted()) {
+      return rows;
+    }
+
+    BufferedExternalSorter.Options sorterOptions = 
BufferedExternalSorter.options();
+    BufferedExternalSorter sorter = 
BufferedExternalSorter.create(sorterOptions);
+    RowCoder rowCoder = RowCoder.of(beamSchema);
+
+    try {
+      for (Row row : rows) {
+        byte[] keyBytes = encodeSortKey(row, sortOrder, icebergSchema, 
beamSchema);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        rowCoder.encode(row, baos);
+        byte[] valBytes = baos.toByteArray();
+        sorter.add(KV.of(keyBytes, valBytes));
+      }
+
+      Iterable<KV<byte[], byte[]>> sortedKVs = sorter.sort();
+      return new Iterable<Row>() {
+        @Override
+        public Iterator<Row> iterator() {
+          final Iterator<KV<byte[], byte[]>> it = sortedKVs.iterator();
+          return new Iterator<Row>() {
+            @Override
+            public boolean hasNext() {
+              return it.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              KV<byte[], byte[]> next = it.next();
+              try {
+                ByteArrayInputStream bais = new 
ByteArrayInputStream(next.getValue());
+                return rowCoder.decode(bais);
+              } catch (IOException e) {
+                throw new RuntimeException("Failed to decode Row during 
sorting", e);
+              }
+            }
+          };
+        }
+      };
+
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to sort rows with external sorter", 
e);
+    }
+  }
+
+  @SuppressWarnings("nullness")
+  public static byte[] encodeSortKey(
+      Row row,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema)
+      throws IOException {
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    for (SortField field : sortOrder.fields()) {
+      String colName = icebergSchema.findColumnName(field.sourceId());
+      Object val = row.getValue(colName);
+
+      if (!field.transform().isIdentity()) {
+        Object icebergVal =
+            IcebergUtils.beamRowToIcebergRecord(icebergSchema, 
row).getField(colName);

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   `IcebergUtils.beamRowToIcebergRecord` is a computationally expensive 
operation as it creates a new `GenericRecord` and copies all fields from the 
Beam `Row`. Calling this inside a loop for every transformed sort field for 
every row will cause severe performance degradation. You should convert the row 
at most once per row, or ideally, only extract and convert the specific fields 
required for sorting.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sorter.BufferedExternalSorter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.SortField;
+import org.apache.iceberg.SortOrder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.ReadableInstant;
+
+/**
+ * A utility class to sort Beam {@link Row}s based on an Iceberg {@link 
SortOrder}. Leverages {@link
+ * BufferedExternalSorter} to spill to local disk when elements exceed memory 
limit.
+ */
+class IcebergRowSorter implements Serializable {
+
+  public static Iterable<Row> sortRows(
+      Iterable<Row> rows,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema) {
+
+    if (sortOrder == null || !sortOrder.isSorted()) {
+      return rows;
+    }
+
+    BufferedExternalSorter.Options sorterOptions = 
BufferedExternalSorter.options();
+    BufferedExternalSorter sorter = 
BufferedExternalSorter.create(sorterOptions);
+    RowCoder rowCoder = RowCoder.of(beamSchema);
+
+    try {
+      for (Row row : rows) {
+        byte[] keyBytes = encodeSortKey(row, sortOrder, icebergSchema, 
beamSchema);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        rowCoder.encode(row, baos);
+        byte[] valBytes = baos.toByteArray();
+        sorter.add(KV.of(keyBytes, valBytes));
+      }
+
+      Iterable<KV<byte[], byte[]>> sortedKVs = sorter.sort();
+      return new Iterable<Row>() {
+        @Override
+        public Iterator<Row> iterator() {
+          final Iterator<KV<byte[], byte[]>> it = sortedKVs.iterator();
+          return new Iterator<Row>() {
+            @Override
+            public boolean hasNext() {
+              return it.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              KV<byte[], byte[]> next = it.next();
+              try {
+                ByteArrayInputStream bais = new 
ByteArrayInputStream(next.getValue());
+                return rowCoder.decode(bais);
+              } catch (IOException e) {
+                throw new RuntimeException("Failed to decode Row during 
sorting", e);
+              }
+            }
+          };
+        }
+      };
+
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to sort rows with external sorter", 
e);
+    }
+  }
+
+  @SuppressWarnings("nullness")
+  public static byte[] encodeSortKey(
+      Row row,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema)
+      throws IOException {
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    for (SortField field : sortOrder.fields()) {
+      String colName = icebergSchema.findColumnName(field.sourceId());

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   Looking up the column name by `sourceId` for every field in every row is 
redundant. These mappings should be pre-calculated once before iterating over 
the rows to improve performance.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergRowSorter.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.beam.sdk.io.iceberg;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.Date;
+import java.util.Iterator;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.extensions.sorter.BufferedExternalSorter;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.Row;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.SortField;
+import org.apache.iceberg.SortOrder;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.ReadableInstant;
+
+/**
+ * A utility class to sort Beam {@link Row}s based on an Iceberg {@link 
SortOrder}. Leverages {@link
+ * BufferedExternalSorter} to spill to local disk when elements exceed memory 
limit.
+ */
+class IcebergRowSorter implements Serializable {
+
+  public static Iterable<Row> sortRows(
+      Iterable<Row> rows,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema) {
+
+    if (sortOrder == null || !sortOrder.isSorted()) {
+      return rows;
+    }
+
+    BufferedExternalSorter.Options sorterOptions = 
BufferedExternalSorter.options();
+    BufferedExternalSorter sorter = 
BufferedExternalSorter.create(sorterOptions);
+    RowCoder rowCoder = RowCoder.of(beamSchema);
+
+    try {
+      for (Row row : rows) {
+        byte[] keyBytes = encodeSortKey(row, sortOrder, icebergSchema, 
beamSchema);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        rowCoder.encode(row, baos);
+        byte[] valBytes = baos.toByteArray();
+        sorter.add(KV.of(keyBytes, valBytes));
+      }
+
+      Iterable<KV<byte[], byte[]>> sortedKVs = sorter.sort();
+      return new Iterable<Row>() {
+        @Override
+        public Iterator<Row> iterator() {
+          final Iterator<KV<byte[], byte[]>> it = sortedKVs.iterator();
+          return new Iterator<Row>() {
+            @Override
+            public boolean hasNext() {
+              return it.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              KV<byte[], byte[]> next = it.next();
+              try {
+                ByteArrayInputStream bais = new 
ByteArrayInputStream(next.getValue());
+                return rowCoder.decode(bais);
+              } catch (IOException e) {
+                throw new RuntimeException("Failed to decode Row during 
sorting", e);
+              }
+            }
+          };
+        }
+      };
+
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to sort rows with external sorter", 
e);
+    }
+  }
+
+  @SuppressWarnings("nullness")
+  public static byte[] encodeSortKey(
+      Row row,
+      SortOrder sortOrder,
+      Schema icebergSchema,
+      org.apache.beam.sdk.schemas.Schema beamSchema)
+      throws IOException {
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+    for (SortField field : sortOrder.fields()) {
+      String colName = icebergSchema.findColumnName(field.sourceId());
+      Object val = row.getValue(colName);
+
+      if (!field.transform().isIdentity()) {
+        Object icebergVal =
+            IcebergUtils.beamRowToIcebergRecord(icebergSchema, 
row).getField(colName);
+        if (icebergVal != null) {
+          val = field.transform().apply(icebergVal);
+        } else {
+          val = null;
+        }
+      }
+
+      boolean isNull = (val == null);
+      boolean isDesc = (field.direction() == SortDirection.DESC);
+      boolean nullsFirst = (field.nullOrder() == NullOrder.NULLS_FIRST);
+
+      // Determine correct header prefix to fulfill the NullOrder contracts
+      byte prefixByte;
+      if (isNull) {
+        if (isDesc) {
+          // Descending: High byte keys sort first.
+          // If Nulls First -> Null gets highest byte (0xFF)
+          // If Nulls Last -> Null gets lowest byte (0x00)
+          prefixByte = nullsFirst ? (byte) 0xFF : (byte) 0x00;
+        } else {
+          // Ascending: Low byte keys sort first.
+          // If Nulls First -> Null gets lowest byte (0x00)
+          // If Nulls Last -> Null gets highest byte (0xFF)
+          prefixByte = nullsFirst ? (byte) 0x00 : (byte) 0xFF;
+        }
+      } else {
+        if (isDesc) {
+          // If non-null and Descending, use a neutral value that sits 
opposite to the null byte
+          prefixByte = nullsFirst ? (byte) 0xFE : (byte) 0x01;
+        } else {
+          prefixByte = nullsFirst ? (byte) 0x01 : (byte) 0x00;
+        }
+      }
+
+      baos.write(prefixByte);
+
+      if (!isNull) {
+        byte[] valBytes = encodeValue(val);
+        // Bitwise invert non-null bytes to sort descending lexicographically
+        if (isDesc) {
+          for (int i = 0; i < valBytes.length; i++) {
+            valBytes[i] = (byte) ~valBytes[i];
+          }
+        }
+        baos.write(valBytes);
+      }
+    }
+
+    return baos.toByteArray();
+  }
+
+  @SuppressWarnings("JavaUtilDate")
+  private static byte[] encodeValue(@Nullable Object val) throws IOException {
+    if (val == null) {
+      return new byte[0];
+    }
+    if (val instanceof String) {
+      return encodeString((String) val);
+    } else if (val instanceof Integer) {
+      int v = (Integer) val;
+      return ByteBuffer.allocate(4).putInt(v ^ Integer.MIN_VALUE).array();
+    } else if (val instanceof Long) {
+      long v = (Long) val;
+      return ByteBuffer.allocate(8).putLong(v ^ Long.MIN_VALUE).array();
+    } else if (val instanceof Float) {
+      int bits = Float.floatToIntBits((Float) val);
+      bits = (bits >= 0) ? (bits ^ Integer.MIN_VALUE) : ~bits;
+      return ByteBuffer.allocate(4).putInt(bits).array();
+    } else if (val instanceof Double) {
+      long bits = Double.doubleToLongBits((Double) val);
+      bits = (bits >= 0) ? (bits ^ Long.MIN_VALUE) : ~bits;
+      return ByteBuffer.allocate(8).putLong(bits).array();

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   Allocating a new `ByteBuffer` and a new `byte[]` for every numeric value in 
every row creates significant GC pressure. Consider using bit manipulation 
directly into a reusable buffer or the main `ByteArrayOutputStream` to improve 
throughput.



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java:
##########
@@ -101,11 +104,22 @@ public void processElement(
       IcebergDestination destination = 
dynamicDestinations.instantiateDestination(tableIdentifier);
       WindowedValue<IcebergDestination> windowedDestination =
           WindowedValues.of(destination, window.maxTimestamp(), window, 
paneInfo);
+      Iterator<Row> rowIt = element.getValue().iterator();
+      if (!rowIt.hasNext()) {
+        return;
+      }
+      Row firstRow = rowIt.next();
+      Schema dataSchema = firstRow.getSchema();

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   Consuming the first element of the `Iterable` to extract the schema is 
risky. If the `Iterable` is not re-iterable (which can happen with certain Beam 
runner implementations), the first row will be lost for the subsequent sorting 
and writing steps. It is safer to pass the schema into the `DoFn` or use a 
method that doesn't consume the iterator.



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


Reply via email to