b-rick commented on code in PR #12424: URL: https://github.com/apache/iceberg/pull/12424#discussion_r2095601716
########## flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicKeySelector.java: ########## @@ -0,0 +1,384 @@ +/* + * 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.dynamic; + +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.EqualityFieldKeySelector; +import org.apache.iceberg.flink.sink.NonThrowingKeySelector; +import org.apache.iceberg.flink.sink.PartitionKeySelector; +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.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class DynamicKeySelector implements NonThrowingKeySelector<DynamicKeySelector.Input, Integer> { + private static final Logger LOG = LoggerFactory.getLogger(DynamicKeySelector.class); + + private final int maxWriteParallelism; + private final Cache<SelectorKey, NonThrowingKeySelector<RowData, Integer>> keySelectorCache; + + DynamicKeySelector(int maxCacheSize, int maxWriteParallelism) { + this.maxWriteParallelism = maxWriteParallelism; + this.keySelectorCache = Caffeine.newBuilder().maximumSize(maxCacheSize).build(); + } + + @Override + public Integer getKey(Input input) { + SelectorKey cacheKey = new SelectorKey(input); + return keySelectorCache + .get( + cacheKey, + k -> + getKeySelector( + input.tableName, + input.schema, + input.spec, + input.mode, + input.equalityFields, + input.writeParallelism)) + .getKey(input.rowData); + } + + public NonThrowingKeySelector<RowData, Integer> getKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + DistributionMode mode, + List<String> equalityFields, + int writeParallelism) { + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFields.isEmpty()) { + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + case HASH: + if (equalityFields.isEmpty()) { + if (spec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFields.contains(partitionField.name()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + schema.columns().stream() + .filter(c -> equalityFields.contains(c.name())) + .collect(Collectors.toList())); + } + return partitionKeySelector( + tableName, schema, spec, writeParallelism, maxWriteParallelism); + } + } + + case RANGE: + if (schema.identifierFieldIds().isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return tableKeySelector(tableName, writeParallelism, maxWriteParallelism); + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return equalityFieldKeySelector( + tableName, schema, equalityFields, writeParallelism, maxWriteParallelism); + } + + default: + throw new IllegalArgumentException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + private static NonThrowingKeySelector<RowData, Integer> equalityFieldKeySelector( + String tableName, + Schema schema, + List<String> equalityFields, + int writeParallelism, + int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new EqualityFieldKeySelector( + schema, + FlinkSchemaUtil.convert(schema), + DynamicRecordProcessor.getEqualityFieldIds(equalityFields, schema)), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector<RowData, Integer> partitionKeySelector( + String tableName, + Schema schema, + PartitionSpec spec, + int writeParallelism, + int maxWriteParallelism) { + NonThrowingKeySelector<RowData, String> inner = + new PartitionKeySelector(spec, schema, FlinkSchemaUtil.convert(schema)); + return new TargetLimitedKeySelector( + in -> inner.getKey(in).hashCode(), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + private static NonThrowingKeySelector<RowData, Integer> tableKeySelector( + String tableName, int writeParallelism, int maxWriteParallelism) { + return new TargetLimitedKeySelector( + new RoundRobinKeySelector<>(writeParallelism), + tableName.hashCode(), + writeParallelism, + maxWriteParallelism); + } + + /** + * Generates a new key using the salt as a base, and reduces the target key range of the {@link + * #wrapped} {@link NonThrowingKeySelector} to {@link #writeParallelism}. + */ + private static class TargetLimitedKeySelector + implements NonThrowingKeySelector<RowData, Integer> { + private final NonThrowingKeySelector<RowData, Integer> wrapped; + private final int writeParallelism; + private final int[] distinctKeys; + + @SuppressWarnings("checkstyle:ParameterAssignment") + TargetLimitedKeySelector( + NonThrowingKeySelector<RowData, Integer> wrapped, + int salt, + int writeParallelism, + int maxWriteParallelism) { + if (writeParallelism > maxWriteParallelism) { Review Comment: > You probably want each table's writeParallelism to change dynamically, similarly to how autoscaling works with multiple Iceberg sinks Indeed this is the part I am thinking about in my use-case. I'm quite new to flink in general, so forgive me if I don't make sense! So from my observations: 1. Definitely holds. I see the autoscaling. 2. I guess we don't have this functionality yet. I'd be interested to see if it's possible though! Even still, I'm thinking there might be a bit of a snag with auto scaling and the PartitionKeySelector. Say there are N streams of data, potentially from N kafka-topic partitions, but each topic partition has data that will eventually be written to a single iceberg partition A. The writing tends to be slower than the reading in this case, so we should get backpressure, and as I understand it, flink should autoscale the writing subtask and increase the number of subtasks. The problem is that I think with the PartitionKeySelector all data will be hashed onto the same writing subtask, because they are all getting written to partition A, meaning we scale up the number of subtasks, but only one will be doing any work. Potentially the round robin selector makes more sense in this scenario, however I do still wonder if it's possible to set the max parallelism there to match the auto scaling. -- 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