stevenzwu commented on code in PR #6160: URL: https://github.com/apache/iceberg/pull/6160#discussion_r1143722125
########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/LocalitySplitAssigner.java: ########## @@ -0,0 +1,168 @@ +/* + * 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.source.assigner; + +import java.util.Collection; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A split assigner that assigns splits to subtasks based on the locality of the splits. */ +@Internal +public class LocalitySplitAssigner implements SplitAssigner { + private static final Logger LOG = LoggerFactory.getLogger(LocalitySplitAssigner.class); + + private static final String DEFAULT_HOSTNAME = "hostname"; + private final Map<String, Set<IcebergSourceSplit>> pendingSplits; + private CompletableFuture<Void> availableFuture; + + public LocalitySplitAssigner() { + this.pendingSplits = Maps.newHashMap(); + } + + public LocalitySplitAssigner(Collection<IcebergSourceSplitState> assignerState) { + this.pendingSplits = Maps.newHashMap(); + Stream<IcebergSourceSplit> splits = assignerState.stream().map(IcebergSourceSplitState::split); + addSplits(splits.collect(Collectors.toList())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } + + IcebergSourceSplit split = + hostname == null + ? getIcebergSourceSplits(DEFAULT_HOSTNAME) + : getIcebergSourceSplits(hostname); + LOG.info("Get Iceberg source splits for: {}", hostname); + + return split != null ? GetSplitResult.forSplit(split) : GetSplitResult.unavailable(); + } + + private IcebergSourceSplit getIcebergSourceSplits(String hostname) { + Set<IcebergSourceSplit> icebergSourceSplits = getSplits(hostname); + + if (icebergSourceSplits != null) { + Optional<IcebergSourceSplit> first = icebergSourceSplits.stream().findFirst(); + if (first.isPresent()) { + pendingSplits.values().forEach(splitSet -> splitSet.remove(first.get())); + return first.get(); + } + } + + return null; + } + + private Set<IcebergSourceSplit> getSplits(String hostname) { + Set<IcebergSourceSplit> icebergSourceSplits = pendingSplits.get(hostname); + if (icebergSourceSplits != null) { + if (!icebergSourceSplits.isEmpty()) { + return icebergSourceSplits; + } + } + + pendingSplits.remove(hostname); Review Comment: is this correct to remove the whole entry for the hostname. Can you explain the algorithm in this method? it is not easy to understand the logic here. ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java: ########## @@ -43,6 +42,20 @@ static boolean isLocalityEnabled( return Util.mayHaveBlockLocations(table.io(), table.location()); } + static SplitAssignerFactory createAssignerFactory( + ReadableConfig readableConfig, Boolean exposeLocality) { + SplitAssignerType assignerType = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); + if (assignerType != null) { + return assignerType.factory(); + } + + if (exposeLocality) { + return SplitAssignerType.LOCALITY.factory(); + } + return SplitAssignerType.SIMPLE.factory(); Review Comment: nit: empty line ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/LocalitySplitAssigner.java: ########## @@ -0,0 +1,168 @@ +/* + * 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.source.assigner; + +import java.util.Collection; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A split assigner that assigns splits to subtasks based on the locality of the splits. */ +@Internal +public class LocalitySplitAssigner implements SplitAssigner { + private static final Logger LOG = LoggerFactory.getLogger(LocalitySplitAssigner.class); + + private static final String DEFAULT_HOSTNAME = "hostname"; + private final Map<String, Set<IcebergSourceSplit>> pendingSplits; + private CompletableFuture<Void> availableFuture; + + public LocalitySplitAssigner() { + this.pendingSplits = Maps.newHashMap(); + } + + public LocalitySplitAssigner(Collection<IcebergSourceSplitState> assignerState) { + this.pendingSplits = Maps.newHashMap(); + Stream<IcebergSourceSplit> splits = assignerState.stream().map(IcebergSourceSplitState::split); + addSplits(splits.collect(Collectors.toList())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } + + IcebergSourceSplit split = + hostname == null Review Comment: wondering if hostname can be null for locality aware assigner. should we add a precondition check? ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/LocalitySplitAssigner.java: ########## @@ -0,0 +1,168 @@ +/* + * 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.source.assigner; + +import java.util.Collection; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A split assigner that assigns splits to subtasks based on the locality of the splits. */ +@Internal +public class LocalitySplitAssigner implements SplitAssigner { + private static final Logger LOG = LoggerFactory.getLogger(LocalitySplitAssigner.class); + + private static final String DEFAULT_HOSTNAME = "hostname"; + private final Map<String, Set<IcebergSourceSplit>> pendingSplits; + private CompletableFuture<Void> availableFuture; + + public LocalitySplitAssigner() { + this.pendingSplits = Maps.newHashMap(); + } + + public LocalitySplitAssigner(Collection<IcebergSourceSplitState> assignerState) { + this.pendingSplits = Maps.newHashMap(); + Stream<IcebergSourceSplit> splits = assignerState.stream().map(IcebergSourceSplitState::split); + addSplits(splits.collect(Collectors.toList())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } + + IcebergSourceSplit split = + hostname == null + ? getIcebergSourceSplits(DEFAULT_HOSTNAME) + : getIcebergSourceSplits(hostname); + LOG.info("Get Iceberg source splits for: {}", hostname); + + return split != null ? GetSplitResult.forSplit(split) : GetSplitResult.unavailable(); + } + + private IcebergSourceSplit getIcebergSourceSplits(String hostname) { + Set<IcebergSourceSplit> icebergSourceSplits = getSplits(hostname); + + if (icebergSourceSplits != null) { + Optional<IcebergSourceSplit> first = icebergSourceSplits.stream().findFirst(); + if (first.isPresent()) { + pendingSplits.values().forEach(splitSet -> splitSet.remove(first.get())); + return first.get(); + } + } + + return null; + } + + private Set<IcebergSourceSplit> getSplits(String hostname) { + Set<IcebergSourceSplit> icebergSourceSplits = pendingSplits.get(hostname); + if (icebergSourceSplits != null) { + if (!icebergSourceSplits.isEmpty()) { + return icebergSourceSplits; + } + } + + pendingSplits.remove(hostname); + + return pendingSplits.values().stream() + .filter(splitSet -> splitSet != null && splitSet.size() > 0) + .findFirst() + .orElse(null); + } + + @Override + public void onDiscoveredSplits(Collection<IcebergSourceSplit> splits) { + addSplits(splits); + } + + @Override + public void onUnassignedSplits(Collection<IcebergSourceSplit> splits) { + addSplits(splits); + } + + private synchronized void addSplits(Collection<IcebergSourceSplit> splits) { + if (splits.isEmpty()) { + return; + } + + for (IcebergSourceSplit split : splits) { + String[] hostnames = split.hostnames(); + if (hostnames == null) { + hostnames = new String[] {DEFAULT_HOSTNAME}; + } + + for (String hostname : hostnames) { + pendingSplits.compute( + hostname, + (key, value) -> { + if (value == null) { + return Sets.newHashSet(split); + } else { + value.add(split); + return value; + } + }); + } + } + + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + + @Override + public synchronized Collection<IcebergSourceSplitState> state() { Review Comment: is this correct? the same split can be added multiple times to the map. one for each hostname, right? ########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java: ########## @@ -89,6 +106,7 @@ public String toString() { .add("files", toString(task.files())) .add("fileOffset", fileOffset) .add("recordOffset", recordOffset) + .add("hostname", hostnames) Review Comment: should be `hostnames` ########## flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java: ########## @@ -57,6 +57,12 @@ private SplitHelpers() {} */ public static List<IcebergSourceSplit> createSplitsFromTransientHadoopTable( TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, null); + } + + public static List<IcebergSourceSplit> createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String[] hostname) Review Comment: should be `hostnames`. please check all the references (singular -> plural). -- 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