pvary commented on code in PR #9321: URL: https://github.com/apache/iceberg/pull/9321#discussion_r1434835739
########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java: ########## @@ -0,0 +1,511 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + + private static final RowDataWrapper KEY_0_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_0 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_1_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_1 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_2_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_2 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_3_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_3 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_4_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_4 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_5_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_5 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_6_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_6 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_7_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_7 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_8_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_8 = SORT_KEY.copy(); + private static final RowDataWrapper KEY_9_WRAPPER = + new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + private static final SortKey SORT_KEY_9 = SORT_KEY.copy(); + + static { + SORT_KEY_0.wrap( + KEY_0_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k0"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_1.wrap( + KEY_1_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k1"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_2.wrap( + KEY_2_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k2"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_3.wrap( + KEY_3_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k3"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_4.wrap( + KEY_4_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k4"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_5.wrap( + KEY_5_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k5"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_6.wrap( + KEY_6_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k6"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_7.wrap( + KEY_7_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k7"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_8.wrap( + KEY_8_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k8"), 1, StringData.fromString("2023-06-20")))); + + SORT_KEY_9.wrap( + KEY_9_WRAPPER.wrap( + GenericRowData.of( + StringData.fromString("k9"), 1, StringData.fromString("2023-06-20")))); + } + + // Total weight is 80 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEY_0, + 35L, + SORT_KEY_1, + 23L, + SORT_KEY_2, + 12L, + SORT_KEY_3, + 4L, + SORT_KEY_4, + 1L, + SORT_KEY_5, + 1L, + SORT_KEY_6, + 1L, + SORT_KEY_7, + 1L, + SORT_KEY_8, + 1L, + SORT_KEY_9, + 1L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 10 (=80/8) + Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + ImmutableMap.of( + SORT_KEY_0, + new MapRangePartitioner.KeyAssignment( + new int[] {0, 1, 2, 3}, new long[] {10L, 10L, 10L, 5L}), + SORT_KEY_1, + new MapRangePartitioner.KeyAssignment(new int[] {3, 4, 5}, new long[] {5L, 10L, 8L}), + SORT_KEY_2, + new MapRangePartitioner.KeyAssignment(new int[] {5, 6}, new long[] {2L, 10L}), + SORT_KEY_3, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {4L}), + SORT_KEY_4, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L}), + SORT_KEY_5, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L}), + SORT_KEY_6, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L}), + SORT_KEY_7, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L}), + SORT_KEY_8, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L}), + SORT_KEY_9, + new MapRangePartitioner.KeyAssignment(new int[] {7}, new long[] {1L})); + Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map<Integer, Pair<Long, Integer>> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(10L, 1), + 1, + Pair.of(10L, 1), + 2, + Pair.of(10L, 1), + 3, + Pair.of(10L, 2), + 4, + Pair.of(10L, 1), + 5, + Pair.of(10L, 2), + 6, + Pair.of(10L, 1), + 7, + Pair.of(10L, 7)); + Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults); + } + + @Test + public void testEvenlyDividable20PercentClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 20.0); + int numPartitions = 8; + + // target subtask weight is 10 before close file cost factored in. + // close file cost is 2 = 20% * 10. + // key weights before and after close file cost factored in + // before: 35, 23, 12, 4, 1, 1, 1, 1, 1, 1 + // close-cost: 8, 6, 4, 2, 2, 2, 2, 2, 2, 2 + // after: 43, 29, 16, 6, 3, 3, 3, 3, 3, 3 + // target subtask weight per subtask is 14 (112/8) + Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + ImmutableMap.of( + SORT_KEY_0, + new MapRangePartitioner.KeyAssignment( + new int[] {0, 1, 2, 3}, new long[] {14L, 14L, 14L, 1L}), Review Comment: All in all, I think a single heuristic loop would be great, like removing splits which are smaller than the `closeFileCost`, and adding the recovered cost to the neighbors if any. - target size: 10, - close cost: 2 - key0 [10, 1], key1[3], key2[6,5] We could make this: 1. Key0 [10], key1[3], key[6,5] - simplest 1. Key0 [10], key1[3], key2[7,5] - somewhat better -- 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