pvary commented on code in PR #9321: URL: https://github.com/apache/iceberg/pull/9321#discussion_r1458488742
########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java: ########## @@ -0,0 +1,288 @@ +/* + * 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 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.ArrayUtil; +import org.apache.iceberg.util.Pair; + +/** + * 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. + */ +class MapRangePartitioner implements Partitioner<RowData> { + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator<StructLike> comparator; + private final Map<SortKey, Long> mapStatistics; + private final double closeFileCostInWeightPercentage; + + // 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) { + 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; + } + + @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) { + // haven't learned about the key before. fall back to random selection. + return ThreadLocalRandom.current().nextInt(numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map<SortKey, KeyAssignment> assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + // add one close file cost for each key even if a key with large weight may be assigned to + // multiple subtasks + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, sortedStatsWithCloseFileCost, targetWeightPerSubtaskWithCloseFileCost); + } + + return assignment; + } + + @VisibleForTesting + NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost() { + return sortedStatsWithCloseFileCost; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map<Integer, Pair<Long, Integer>> assignmentInfo() { + Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeights[i]; + Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map<SortKey, KeyAssignment> buildAssignment( + int numPartitions, + NavigableMap<SortKey, Long> sortedStatistics, + long targetWeightPerSubtask) { + Map<SortKey, KeyAssignment> assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List<Integer> assignedSubtasks = Lists.newArrayList(); + List<Long> subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() && subtaskId < numPartitions) { + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + // assign the remaining weight of key to the current subtask if it is the last subtask + // or if the subtask has more capacity than the remaining key weight + if (subtaskId == numPartitions - 1 || keyRemainingWeight < subtaskRemainingWeight) { Review Comment: > the job is constantly restarting Do we store the distribution in the state? That’s the reason why the job will fail again after a restart? -- 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