atognolag commented on code in PR #38406:
URL: https://github.com/apache/beam/pull/38406#discussion_r3203023168


##########
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:
   Now exactly one conversion occurs per row when one or more transformed 
columns are sorted, regardless of the number of sort fields.



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