hililiwei commented on code in PR #6160: URL: https://github.com/apache/iceberg/pull/6160#discussion_r1111756854
########## flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/LocalitySplitAssigner.java: ########## @@ -0,0 +1,165 @@ +/* + * 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.ArrayDeque; +import java.util.Collection; +import java.util.Deque; +import java.util.Iterator; +import java.util.Map; +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; + +@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<Set<String>, Deque<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 GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } + + Deque<IcebergSourceSplit> icebergSourceSplits = + hostname == null + ? getIcebergSourceSplits(DEFAULT_HOSTNAME, pendingSplits) + : getIcebergSourceSplits(hostname, pendingSplits); + LOG.info("Get Iceberg source splits for: {}", hostname); + + if (!icebergSourceSplits.isEmpty()) { + IcebergSourceSplit split = icebergSourceSplits.poll(); + return GetSplitResult.forSplit(split); + } + + return GetSplitResult.unavailable(); + } + + private Deque<IcebergSourceSplit> getIcebergSourceSplits( + String hostname, Map<Set<String>, Deque<IcebergSourceSplit>> splitsDeque) { + if (splitsDeque.isEmpty()) { + return new ArrayDeque<>(); + } + + Iterator<Map.Entry<Set<String>, Deque<IcebergSourceSplit>>> splitsIterator = + splitsDeque.entrySet().iterator(); + while (splitsIterator.hasNext()) { Review Comment: I understand what you mean. Perhaps we could consider using another approach: 1. Construct a `Map<Hostname, Set<IcebergSourceSplit>>` named "`splits`". 2. Add an `IcebergSourceSplit` to "splits". Since an `IcebergSourceSplit` may have multiple hostnames, there will be multiple keys in "`splits`" that point to a Set containing the same `IcebergSourceSplit`. 3. To retrieve a split for a specific hostname in "`splits`", retrieve all `Sets` pointed to by the split's hostnames and remove the split from each of them. 4. Return the split. Here's some pseudocode: ``` Map<Hostname, Set<IcebergSourceSplit>> splits = new ... // add IcebergSourceSplit splitA: splitA.hostnames.forEach{ splits.put(hostname, new Set(splitA)) or splits.get(hostname).add(splitA) } // get IcebergSourceSplit splitB: IcebergSourceSplit splitB = splits.get("host1").poll() splitB.hostnames.forEach{ splits.get(hostname).remove(splitB) } return splitB; ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
