pvary commented on code in PR #9321:
URL: https://github.com/apache/iceberg/pull/9321#discussion_r1524300363


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java:
##########
@@ -0,0 +1,373 @@
+/*
+ * 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.flink.sink.shuffle;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortKey;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderComparators;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataWrapper;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Internal partitioner implementation that supports MapDataStatistics, which 
is typically used for
+ * low-cardinality use cases. While MapDataStatistics can keep accurate 
counters, it can't be used
+ * for high-cardinality use cases. Otherwise, the memory footprint is too high.
+ *
+ * <p>It is a greedy algorithm for bin packing. With close file cost, the 
calculation isn't always
+ * precise when calculating close cost for every file, target weight per 
subtask, padding residual
+ * weight, assigned weight without close cost.
+ *
+ * <p>All actions should be executed in a single Flink mailbox thread. So 
there is no need to make
+ * it thread safe.
+ */
+class MapRangePartitioner implements Partitioner<RowData> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(MapRangePartitioner.class);
+
+  private final RowDataWrapper rowDataWrapper;
+  private final SortKey sortKey;
+  private final Comparator<StructLike> comparator;
+  private final Map<SortKey, Long> mapStatistics;
+  private final double closeFileCostInWeightPercentage;
+
+  // Counter that tracks how many times a new key encountered
+  // where there is no traffic statistics learned about it.
+  private long newSortKeyCounter;
+  private long lastNewSortKeyLogTimeMilli;
+
+  // lazily computed due to the need of numPartitions
+  private Map<SortKey, KeyAssignment> assignment;
+  private NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost;
+
+  MapRangePartitioner(
+      Schema schema,
+      SortOrder sortOrder,
+      MapDataStatistics dataStatistics,
+      double closeFileCostInWeightPercentage) {
+    dataStatistics
+        .statistics()
+        .entrySet()
+        .forEach(
+            entry ->
+                Preconditions.checkArgument(
+                    entry.getValue() > 0,
+                    "Invalid statistics: weight is 0 for key %s",
+                    entry.getKey()));
+
+    this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), 
schema.asStruct());
+    this.sortKey = new SortKey(schema, sortOrder);
+    this.comparator = SortOrderComparators.forSchema(schema, sortOrder);
+    this.mapStatistics = dataStatistics.statistics();
+    this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage;
+    this.newSortKeyCounter = 0;
+    this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis();
+  }
+
+  @Override
+  public int partition(RowData row, int numPartitions) {
+    // assignment table can only be built lazily when first referenced here,
+    // because number of partitions (downstream subtasks) is needed
+    Map<SortKey, KeyAssignment> assignmentMap = assignment(numPartitions);
+    // reuse the sortKey and rowDataWrapper
+    sortKey.wrap(rowDataWrapper.wrap(row));
+    KeyAssignment keyAssignment = assignmentMap.get(sortKey);
+    if (keyAssignment == null) {
+      LOG.trace(

Review Comment:
   Maybe increase to debug level? Or once per calculation in ifo level? This is 
quite important information, because we are working suboptimally



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java:
##########
@@ -0,0 +1,373 @@
+/*
+ * 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.flink.sink.shuffle;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortKey;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.SortOrderComparators;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataWrapper;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Internal partitioner implementation that supports MapDataStatistics, which 
is typically used for
+ * low-cardinality use cases. While MapDataStatistics can keep accurate 
counters, it can't be used
+ * for high-cardinality use cases. Otherwise, the memory footprint is too high.
+ *
+ * <p>It is a greedy algorithm for bin packing. With close file cost, the 
calculation isn't always
+ * precise when calculating close cost for every file, target weight per 
subtask, padding residual
+ * weight, assigned weight without close cost.
+ *
+ * <p>All actions should be executed in a single Flink mailbox thread. So 
there is no need to make
+ * it thread safe.
+ */
+class MapRangePartitioner implements Partitioner<RowData> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(MapRangePartitioner.class);
+
+  private final RowDataWrapper rowDataWrapper;
+  private final SortKey sortKey;
+  private final Comparator<StructLike> comparator;
+  private final Map<SortKey, Long> mapStatistics;
+  private final double closeFileCostInWeightPercentage;
+
+  // Counter that tracks how many times a new key encountered
+  // where there is no traffic statistics learned about it.
+  private long newSortKeyCounter;
+  private long lastNewSortKeyLogTimeMilli;
+
+  // lazily computed due to the need of numPartitions
+  private Map<SortKey, KeyAssignment> assignment;
+  private NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost;
+
+  MapRangePartitioner(
+      Schema schema,
+      SortOrder sortOrder,
+      MapDataStatistics dataStatistics,
+      double closeFileCostInWeightPercentage) {
+    dataStatistics
+        .statistics()
+        .entrySet()
+        .forEach(
+            entry ->
+                Preconditions.checkArgument(
+                    entry.getValue() > 0,
+                    "Invalid statistics: weight is 0 for key %s",
+                    entry.getKey()));
+
+    this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), 
schema.asStruct());
+    this.sortKey = new SortKey(schema, sortOrder);
+    this.comparator = SortOrderComparators.forSchema(schema, sortOrder);
+    this.mapStatistics = dataStatistics.statistics();
+    this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage;
+    this.newSortKeyCounter = 0;
+    this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis();
+  }
+
+  @Override
+  public int partition(RowData row, int numPartitions) {
+    // assignment table can only be built lazily when first referenced here,
+    // because number of partitions (downstream subtasks) is needed
+    Map<SortKey, KeyAssignment> assignmentMap = assignment(numPartitions);
+    // reuse the sortKey and rowDataWrapper
+    sortKey.wrap(rowDataWrapper.wrap(row));
+    KeyAssignment keyAssignment = assignmentMap.get(sortKey);
+    if (keyAssignment == null) {
+      LOG.trace(

Review Comment:
   Maybe increase to debug level? Or once per calculation in info level? This 
is quite important information, because we are working suboptimally



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to