EricJoy2048 commented on code in PR #6161: URL: https://github.com/apache/seatunnel/pull/6161#discussion_r1448241762
########## seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/source/split/DorisSourceSplitEnumerator.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.seatunnel.connectors.doris.source.split; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCodeDeprecated; +import org.apache.seatunnel.connectors.doris.config.DorisConfig; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; +import org.apache.seatunnel.connectors.doris.rest.PartitionDefinition; +import org.apache.seatunnel.connectors.doris.rest.RestService; +import org.apache.seatunnel.connectors.doris.source.DorisSourceState; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +@Slf4j +public class DorisSourceSplitEnumerator + implements SourceSplitEnumerator<DorisSourceSplit, DorisSourceState> { + + private Context<DorisSourceSplit> context; + private DorisConfig dorisConfig; + + private volatile boolean shouldEnumerate; + + private final Map<Integer, List<DorisSourceSplit>> pendingSplit; + + private SeaTunnelRowType seaTunnelRowType; + private final Object stateLock = new Object(); + + public DorisSourceSplitEnumerator( + Context<DorisSourceSplit> context, + DorisConfig dorisConfig, + SeaTunnelRowType seaTunnelRowType) { + this(context, dorisConfig, seaTunnelRowType, null); + } + + public DorisSourceSplitEnumerator( + Context<DorisSourceSplit> context, + DorisConfig dorisConfig, + SeaTunnelRowType rowType, + DorisSourceState dorisSourceState) { + this.context = context; + this.dorisConfig = dorisConfig; + this.seaTunnelRowType = rowType; + this.pendingSplit = new ConcurrentHashMap<>(); + this.shouldEnumerate = (dorisSourceState == null); + if (dorisSourceState != null) { + this.shouldEnumerate = dorisSourceState.isShouldEnumerate(); + this.pendingSplit.putAll(dorisSourceState.getPendingSplit()); + } + } + + @Override + public void open() {} + + @Override + public void close() throws IOException {} + + @Override + public void run() { + Set<Integer> readers = context.registeredReaders(); + if (shouldEnumerate) { + List<DorisSourceSplit> dorisSourceSplits = getDorisSourceSplit(); + synchronized (stateLock) { + addPendingSplit(dorisSourceSplits); + shouldEnumerate = false; + assignSplit(readers); + } + } + + log.debug( + "No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); + readers.forEach(context::signalNoMoreSplits); + } + + @Override + public void addSplitsBack(List<DorisSourceSplit> splits, int subtaskId) { + log.debug("Add back splits {} to DorisSourceSplitEnumerator.", splits); + if (!splits.isEmpty()) { + synchronized (stateLock) { + addPendingSplit(splits); + assignSplit(Collections.singletonList(subtaskId)); + } + } + } + + @Override + public int currentUnassignedSplitSize() { + return this.pendingSplit.size(); + } + + @Override + public void handleSplitRequest(int subtaskId) { + throw new DorisConnectorException( + CommonErrorCodeDeprecated.UNSUPPORTED_OPERATION, + String.format("Unsupported handleSplitRequest: %d", subtaskId)); + } + + @Override + public void registerReader(int subtaskId) { + log.debug("Register reader {} to DorisSourceSplitEnumerator.", subtaskId); + if (!pendingSplit.isEmpty()) { + synchronized (stateLock) { + assignSplit(Collections.singletonList(subtaskId)); + } + } + } + + @Override + public DorisSourceState snapshotState(long checkpointId) { + synchronized (stateLock) { + return new DorisSourceState(shouldEnumerate, pendingSplit); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + private List<DorisSourceSplit> getDorisSourceSplit() { + List<DorisSourceSplit> splits = new ArrayList<>(); + List<PartitionDefinition> partitions = + RestService.findPartitions(seaTunnelRowType, dorisConfig, log); + for (PartitionDefinition partition : partitions) { + splits.add(new DorisSourceSplit(partition, String.valueOf(partition.hashCode()))); + } + return splits; + } + + private void addPendingSplit(Collection<DorisSourceSplit> splits) { + int readerCount = context.currentParallelism(); + for (DorisSourceSplit split : splits) { + int ownerReader = getSplitOwner(split.splitId(), readerCount); + log.info("Assigning {} to {} reader.", split.splitId(), ownerReader); + pendingSplit.computeIfAbsent(ownerReader, f -> new ArrayList<>()).add(split); + } + } + + private static int getSplitOwner(String tp, int numReaders) { + return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; + } + + private void assignSplit(Collection<Integer> readers) { + for (Integer reader : readers) { + final List<DorisSourceSplit> assignmentForReader = pendingSplit.remove(reader); + + if (assignmentForReader != null && !assignmentForReader.isEmpty()) { + + String splitsInfo = + assignmentForReader.stream() + .map(DorisSourceSplit::getSplitId) + .collect(Collectors.joining(",")); + log.info("Assign splits {} to reader {}", splitsInfo, reader); + try { + context.assignSplit(reader, assignmentForReader); + } catch (Exception e) { + log.error( + "Failed to assign splits {} to reader {}", + assignmentForReader, + reader, + e); + pendingSplit.put(reader, assignmentForReader); Review Comment: This will cause split lost. `readers.forEach(context::signalNoMoreSplits);` will be call after method `assignSplit` return. But maybe some split added to `pendingSplit ` because `context.assignSplit(reader, assignmentForReader);` throw Exception. ``` public void run() { Set<Integer> readers = context.registeredReaders(); if (shouldEnumerate) { List<DorisSourceSplit> dorisSourceSplits = getDorisSourceSplit(); synchronized (stateLock) { addPendingSplit(dorisSourceSplits); shouldEnumerate = false; assignSplit(readers); } } log.debug( "No more splits to assign." + " Sending NoMoreSplitsEvent to reader {}.", readers); readers.forEach(context::signalNoMoreSplits); } ``` -- 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]
