diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGate.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGate.java new file mode 100644 index 00000000000..dd5424141e1 --- /dev/null +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGate.java @@ -0,0 +1,183 @@ +/* + * 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.celeborn.plugin.flink; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.event.TaskEvent; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.SupplierWithException; + +import org.apache.celeborn.common.CelebornConf; + +/** An abstract {@link IndexedInputGate} which ingest data from remote shuffle workers. */ +public abstract class AbstractRemoteShuffleInputGate extends IndexedInputGate { + + public final RemoteShuffleInputGateDelegation inputGateDelegation; + + public AbstractRemoteShuffleInputGate( + CelebornConf celebornConf, + String taskName, + int gateIndex, + InputGateDeploymentDescriptor gateDescriptor, + SupplierWithException bufferPoolFactory, + BufferDecompressor bufferDecompressor, + int numConcurrentReading) { + inputGateDelegation = + new RemoteShuffleInputGateDelegation( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading, + availabilityHelper, + gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), + gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); + } + + /** Setup gate and build network connections. */ + @Override + public void setup() throws IOException { + inputGateDelegation.setup(); + } + + /** Index of the gate of the corresponding computing task. */ + @Override + public int getGateIndex() { + return inputGateDelegation.getGateIndex(); + } + + /** Get number of input channels. A channel is a data flow from one shuffle worker. */ + @Override + public int getNumberOfInputChannels() { + return inputGateDelegation.getBufferReaders().size(); + } + + /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ + @Override + public boolean isFinished() { + return inputGateDelegation.isFinished(); + } + + @Override + public Optional getNext() { + throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); + } + + /** Poll a received {@link BufferOrEvent}. */ + @Override + public Optional pollNext() throws IOException { + return inputGateDelegation.pollNext(); + } + + /** Close all reading channels inside this {@link AbstractRemoteShuffleInputGate}. */ + @Override + public void close() throws Exception { + inputGateDelegation.close(); + } + + /** Get {@link InputChannelInfo}s of this {@link AbstractRemoteShuffleInputGate}. */ + @Override + public List getChannelInfos() { + return inputGateDelegation.getChannelsInfo(); + } + + @Override + public void requestPartitions() { + // do-nothing + } + + @Override + public void checkpointStarted(CheckpointBarrier barrier) { + // do-nothing. + } + + @Override + public void checkpointStopped(long cancelledCheckpointId) { + // do-nothing. + } + + @Override + public void triggerDebloating() { + // do-nothing. + } + + @Override + public List getUnfinishedChannels() { + return Collections.emptyList(); + } + + @Override + public EndOfDataStatus hasReceivedEndOfData() { + if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { + return EndOfDataStatus.NOT_END_OF_DATA; + } else { + // Keep compatibility with streaming mode. + return EndOfDataStatus.DRAINED; + } + } + + @Override + public void finishReadRecoveredState() { + // do-nothing. + } + + @Override + public abstract InputChannel getChannel(int channelIndex); + + @Override + public void sendTaskEvent(TaskEvent event) { + throw new FlinkRuntimeException("Method should not be called."); + } + + @Override + public void resumeConsumption(InputChannelInfo channelInfo) { + throw new FlinkRuntimeException("Method should not be called."); + } + + @Override + public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} + + @Override + public CompletableFuture getStateConsumedFuture() { + return CompletableFuture.completedFuture(null); + } + + @Override + public String toString() { + return String.format( + "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", + inputGateDelegation.getTaskName(), + inputGateDelegation.getGateIndex(), + inputGateDelegation.getGateDescriptor().toString()); + } +} diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java index b6ee418d090..9568ac5461c 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleInputGateFactory.java @@ -32,7 +32,7 @@ import org.apache.celeborn.common.CelebornConf; import org.apache.celeborn.plugin.flink.utils.Utils; -/** Factory class to create RemoteShuffleInputGate. */ +/** Factory class to create {@link AbstractRemoteShuffleInputGate}. */ public abstract class AbstractRemoteShuffleInputGateFactory { public static final int MIN_BUFFERS_PER_GATE = 16; diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartition.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartition.java new file mode 100644 index 00000000000..bafc4ffcc53 --- /dev/null +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartition.java @@ -0,0 +1,240 @@ +/* + * 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.celeborn.plugin.flink; + +import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; +import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +import javax.annotation.Nullable; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.network.api.EndOfData; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.api.StopMode; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; +import org.apache.flink.runtime.io.network.buffer.BufferCompressor; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.util.function.SupplierWithException; + +import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; +import org.apache.celeborn.plugin.flink.buffer.DataBuffer; +import org.apache.celeborn.plugin.flink.utils.BufferUtils; +import org.apache.celeborn.plugin.flink.utils.Utils; + +/** + * An abstract {@link ResultPartition} which appends records and events to {@link DataBuffer} and + * after the {@link DataBuffer} is full, all data in the {@link DataBuffer} will be copied and + * spilled to the remote shuffle service in subpartition index order sequentially. Large records + * that can not be appended to an empty {@link DataBuffer} will be spilled directly. + */ +public abstract class AbstractRemoteShuffleResultPartition extends ResultPartition { + + private final RemoteShuffleResultPartitionDelegation delegation; + + private final SupplierWithException bufferPoolFactory; + + public AbstractRemoteShuffleResultPartition( + String owningTaskName, + int partitionIndex, + ResultPartitionID partitionId, + ResultPartitionType partitionType, + int numSubpartitions, + int numTargetKeyGroups, + int networkBufferSize, + ResultPartitionManager partitionManager, + @Nullable BufferCompressor bufferCompressor, + SupplierWithException bufferPoolFactory, + RemoteShuffleOutputGate outputGate) { + super( + owningTaskName, + partitionIndex, + partitionId, + partitionType, + numSubpartitions, + numTargetKeyGroups, + partitionManager, + bufferCompressor, + bufferPoolFactory); + delegation = + new RemoteShuffleResultPartitionDelegation( + networkBufferSize, outputGate, this::updateStatistics, numSubpartitions); + this.bufferPoolFactory = bufferPoolFactory; + } + + @Override + public void setup() throws IOException { + // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. + // The reason is that this partition will be registered to the partition manager during + // `super.setup()`. + // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), + // Flink does not trigger the resource releasing over TM. Therefore, the partition object is + // leaked. + // So we copy the logic of `setup` but don't register partition to partition manager. + checkState( + this.bufferPool == null, + "Bug in result partition setup logic: Already registered buffer pool."); + this.bufferPool = checkNotNull(bufferPoolFactory.get()); + // this is an empty method, but still call it in case of we implement it in the future. + setupInternal(); + BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); + delegation.setup( + bufferPool, bufferCompressor, this::canBeCompressed, this::checkInProduceState); + } + + @Override + protected void setupInternal() { + // do not need to implement + } + + @Override + public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { + delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); + } + + @Override + public void broadcastRecord(ByteBuffer record) throws IOException { + delegation.broadcast(record, DataType.DATA_BUFFER); + } + + @Override + public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { + Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); + try { + ByteBuffer serializedEvent = buffer.getNioBufferReadable(); + delegation.broadcast(serializedEvent, buffer.getDataType()); + } finally { + buffer.recycleBuffer(); + } + } + + @Override + public void alignedBarrierTimeout(long l) {} + + @Override + public void abortCheckpoint(long l, CheckpointException e) {} + + @Override + public void finish() throws IOException { + Utils.checkState(!isReleased(), "Result partition is already released."); + broadcastEvent(EndOfPartitionEvent.INSTANCE, false); + delegation.finish(); + super.finish(); + } + + @Override + public synchronized void close() { + delegation.close(super::close); + } + + @Override + protected void releaseInternal() { + // no-op + } + + @Override + public void flushAll() { + delegation.flushAll(); + } + + @Override + public void flush(int subpartitionIndex) { + flushAll(); + } + + @Override + public CompletableFuture getAvailableFuture() { + return AVAILABLE; + } + + @Override + public int getNumberOfQueuedBuffers() { + return 0; + } + + @Override + public long getSizeOfQueuedBuffersUnsafe() { + return 0; + } + + @Override + public int getNumberOfQueuedBuffers(int targetSubpartition) { + return 0; + } + + @Override + public ResultSubpartitionView createSubpartitionView( + int index, BufferAvailabilityListener availabilityListener) { + throw new UnsupportedOperationException("Not supported."); + } + + @Override + public void notifyEndOfData(StopMode mode) throws IOException { + if (!delegation.isEndOfDataNotified()) { + broadcastEvent(new EndOfData(mode), false); + delegation.setEndOfDataNotified(true); + } + } + + @Override + public CompletableFuture getAllDataProcessedFuture() { + return CompletableFuture.completedFuture(null); + } + + @Override + public String toString() { + return "ResultPartition " + + partitionId.toString() + + " [" + + partitionType + + ", " + + numSubpartitions + + " subpartitions, shuffle-descriptor: " + + delegation.getOutputGate().getShuffleDesc() + + "]"; + } + + @VisibleForTesting + public RemoteShuffleResultPartitionDelegation getDelegation() { + return delegation; + } + + public void updateStatistics(BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast) { + numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); + long readableBytes = + (long) bufferWithSubpartition.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + updateReadableBytes(bufferWithSubpartition, isBroadcast, readableBytes); + numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); + } + + public abstract void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes); +} diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java index 58a4a36d9b2..dfde3478799 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleResultPartitionFactory.java @@ -38,7 +38,7 @@ import org.apache.celeborn.common.CelebornConf; import org.apache.celeborn.plugin.flink.utils.Utils; -/** Factory class to create {@link RemoteShuffleResultPartition}. */ +/** Factory class to create {@link AbstractRemoteShuffleResultPartition}. */ public abstract class AbstractRemoteShuffleResultPartitionFactory { private static final Logger LOG = diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleServiceFactory.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleServiceFactory.java index 8927de331b4..a058e4b892e 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleServiceFactory.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleServiceFactory.java @@ -26,16 +26,39 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; +import org.apache.flink.runtime.shuffle.ShuffleMaster; +import org.apache.flink.runtime.shuffle.ShuffleMasterContext; +import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.flink.runtime.util.ConfigurationParserUtils; import org.apache.celeborn.common.CelebornConf; import org.apache.celeborn.plugin.flink.utils.FlinkUtils; -public class AbstractRemoteShuffleServiceFactory { - public static int calculateNumberOfNetworkBuffers(MemorySize memorySize, int bufferSize) { +public abstract class AbstractRemoteShuffleServiceFactory + implements ShuffleServiceFactory { + + public final NettyShuffleServiceFactory nettyShuffleServiceFactory = + new NettyShuffleServiceFactory(); + + @Override + public ShuffleMaster createShuffleMaster( + ShuffleMasterContext shuffleMasterContext) { + return new RemoteShuffleMaster(shuffleMasterContext, nettyShuffleServiceFactory); + } + + @Override + public abstract ShuffleEnvironment + createShuffleEnvironment(ShuffleEnvironmentContext shuffleEnvironmentContext); + + private int calculateNumberOfNetworkBuffers(MemorySize memorySize, int bufferSize) { long numBuffersLong = memorySize.getBytes() / bufferSize; if (numBuffersLong > Integer.MAX_VALUE) { throw new IllegalArgumentException( @@ -46,7 +69,7 @@ public static int calculateNumberOfNetworkBuffers(MemorySize memorySize, int buf return (int) numBuffersLong; } - protected static AbstractRemoteShuffleServiceParameters initializePreCreateShuffleEnvironment( + protected AbstractRemoteShuffleServiceParameters initializePreCreateShuffleEnvironment( ShuffleEnvironmentContext shuffleEnvironmentContext) { Configuration configuration = shuffleEnvironmentContext.getConfiguration(); int bufferSize = ConfigurationParserUtils.getPageSize(configuration); diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleEnvironment.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java similarity index 63% rename from client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleEnvironment.java rename to client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java index ed665c27c28..0ae7e9c8b7d 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/AbstractRemoteShuffleEnvironment.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java @@ -20,12 +20,15 @@ import static org.apache.celeborn.plugin.flink.metric.RemoteShuffleMetricFactory.createShuffleIOOwnerMetricGroup; import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; -import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.*; +import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_INPUT; +import static org.apache.flink.runtime.io.network.metrics.NettyShuffleMetricFactory.METRIC_GROUP_OUTPUT; import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.MetricGroup; @@ -35,10 +38,13 @@ import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; import org.apache.flink.util.FlinkRuntimeException; @@ -46,43 +52,64 @@ import org.slf4j.LoggerFactory; import org.apache.celeborn.common.CelebornConf; +import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; /** * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing * shuffle environment on flink TM side. */ -public abstract class AbstractRemoteShuffleEnvironment { +public class RemoteShuffleEnvironment + implements ShuffleEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(AbstractRemoteShuffleEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(RemoteShuffleEnvironment.class); /** Network buffer pool for shuffle read and shuffle write. */ - protected final NetworkBufferPool networkBufferPool; + private final NetworkBufferPool networkBufferPool; /** A trivial {@link ResultPartitionManager}. */ - protected final ResultPartitionManager resultPartitionManager; + private final ResultPartitionManager resultPartitionManager; + + /** Factory class to create {@link AbstractRemoteShuffleResultPartition}. */ + private final AbstractRemoteShuffleResultPartitionFactory resultPartitionFactory; + + private final AbstractRemoteShuffleInputGateFactory inputGateFactory; + + private final CelebornConf conf; + + private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; /** Whether the shuffle environment is closed. */ - protected boolean isClosed; + private boolean isClosed; + + private final Object lock = new Object(); - protected final Object lock = new Object(); + private final ConcurrentHashMap.KeySetView nettyResultIds = + ConcurrentHashMap.newKeySet(); - protected final CelebornConf conf; + private final ConcurrentHashMap.KeySetView + nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); /** * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. * @param resultPartitionManager A trivial {@link ResultPartitionManager}. */ - public AbstractRemoteShuffleEnvironment( + public RemoteShuffleEnvironment( NetworkBufferPool networkBufferPool, ResultPartitionManager resultPartitionManager, - CelebornConf conf) { - + AbstractRemoteShuffleResultPartitionFactory resultPartitionFactory, + AbstractRemoteShuffleInputGateFactory inputGateFactory, + CelebornConf conf, + NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper) { this.networkBufferPool = networkBufferPool; this.resultPartitionManager = resultPartitionManager; + this.resultPartitionFactory = resultPartitionFactory; + this.inputGateFactory = inputGateFactory; this.conf = conf; + this.shuffleEnvironmentWrapper = shuffleEnvironmentWrapper; this.isClosed = false; } + @Override public void close() { LOG.info("Close RemoteShuffleEnvironment."); synchronized (lock) { @@ -105,6 +132,7 @@ public void close() { } } + @Override public int start() throws IOException { synchronized (lock) { checkState(!isClosed, "The RemoteShuffleEnvironment has already been shut down."); @@ -114,10 +142,12 @@ public int start() throws IOException { } } + @Override public boolean updatePartitionInfo(ExecutionAttemptID consumerID, PartitionInfo partitionInfo) { throw new FlinkRuntimeException("Not implemented yet."); } + @Override public ShuffleIOOwnerContext createShuffleIOOwnerContext( String ownerName, ExecutionAttemptID executionAttemptID, MetricGroup parentGroup) { MetricGroup remoteGroup = createShuffleIOOwnerMetricGroup(checkNotNull(parentGroup)); @@ -129,15 +159,12 @@ public ShuffleIOOwnerContext createShuffleIOOwnerContext( remoteGroup.addGroup(METRIC_GROUP_INPUT)); } + @Override public Collection getPartitionsOccupyingLocalResources() { return resultPartitionManager.getUnreleasedPartitions(); } - @VisibleForTesting - NetworkBufferPool getNetworkBufferPool() { - return networkBufferPool; - } - + @Override public List createResultPartitionWriters( ShuffleIOOwnerContext ownerContext, List resultPartitionDeploymentDescriptors) { @@ -156,12 +183,38 @@ public List createResultPartitionWriters( } } - public abstract ResultPartitionWriter createResultPartitionWriterInternal( + private ResultPartitionWriter createResultPartitionWriterInternal( ShuffleIOOwnerContext ownerContext, int index, ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf); + CelebornConf conf) { + if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() + instanceof RemoteShuffleDescriptor) { + return resultPartitionFactory.create( + ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); + } else { + nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); + nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); + return shuffleEnvironmentWrapper + .nettyResultPartitionFactory() + .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); + } + } + + @Override + public void releasePartitionsLocally(Collection partitionIds) { + List resultPartitionIds = + partitionIds.stream() + .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) + .collect(Collectors.toList()); + if (!resultPartitionIds.isEmpty()) { + shuffleEnvironmentWrapper + .nettyShuffleEnvironment() + .releasePartitionsLocally(resultPartitionIds); + } + } + @Override public List createInputGates( ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider producerStateProvider, @@ -180,9 +233,26 @@ public List createInputGates( } } - abstract IndexedInputGate createInputGateInternal( + private IndexedInputGate createInputGateInternal( ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider producerStateProvider, int gateIndex, - InputGateDeploymentDescriptor igdd); + InputGateDeploymentDescriptor igdd) { + return nettyResultIds.contains(igdd.getConsumedResultId()) + ? shuffleEnvironmentWrapper + .nettyInputGateFactory() + .create( + ownerContext, + gateIndex, + igdd, + producerStateProvider, + new InputChannelMetrics( + ownerContext.getInputGroup(), ownerContext.getParentGroup())) + : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); + } + + @VisibleForTesting + AbstractRemoteShuffleResultPartitionFactory getResultPartitionFactory() { + return resultPartitionFactory; + } } diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java index 35ef341ebb0..cb1a9e7f9d9 100644 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java +++ b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleMaster.java @@ -74,20 +74,17 @@ public class RemoteShuffleMaster implements ShuffleMaster { private final ScheduledExecutorService executor = ThreadUtils.newDaemonSingleThreadScheduledExecutor( "celeborn-client-remote-shuffle-master-executor"); - private final ResultPartitionAdapter resultPartitionDelegation; private final long lifecycleManagerTimestamp; private final NettyShuffleServiceFactory nettyShuffleServiceFactory; private volatile NettyShuffleMaster nettyShuffleMaster; public RemoteShuffleMaster( ShuffleMasterContext shuffleMasterContext, - ResultPartitionAdapter resultPartitionDelegation, @Nullable NettyShuffleServiceFactory nettyShuffleServiceFactory) { Configuration configuration = shuffleMasterContext.getConfiguration(); checkShuffleConfig(configuration); this.conf = FlinkUtils.toCelebornConf(configuration); this.shuffleMasterContext = shuffleMasterContext; - this.resultPartitionDelegation = resultPartitionDelegation; this.lifecycleManagerTimestamp = System.currentTimeMillis(); this.nettyShuffleServiceFactory = nettyShuffleServiceFactory; } @@ -253,8 +250,7 @@ public MemorySize computeShuffleMemorySizeForTask( TaskInputsOutputsDescriptor taskInputsOutputsDescriptor) { for (ResultPartitionType partitionType : taskInputsOutputsDescriptor.getPartitionTypes().values()) { - boolean isBlockingShuffle = - resultPartitionDelegation.isBlockingResultPartition(partitionType); + boolean isBlockingShuffle = partitionType.isBlockingOrBlockingPersistentResultPartition(); if (!isBlockingShuffle) { throw new RuntimeException( "Blocking result partition type expected but found " + partitionType); diff --git a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/ResultPartitionAdapter.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/ResultPartitionAdapter.java deleted file mode 100644 index 6ee9daf62c9..00000000000 --- a/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/ResultPartitionAdapter.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public interface ResultPartitionAdapter { - boolean isBlockingResultPartition(ResultPartitionType partitionType); -} diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java b/client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java similarity index 100% rename from client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java rename to client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java deleted file mode 100644 index 844fd276a99..00000000000 --- a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; -import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.ShuffleEnvironment; -import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; - -import org.apache.celeborn.common.CelebornConf; -import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; - -/** - * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing - * shuffle environment on flink TM side. - */ -public class RemoteShuffleEnvironment extends AbstractRemoteShuffleEnvironment - implements ShuffleEnvironment { - - /** Factory class to create {@link RemoteShuffleResultPartition}. */ - private final RemoteShuffleResultPartitionFactory resultPartitionFactory; - - private final RemoteShuffleInputGateFactory inputGateFactory; - - private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; - - private final ConcurrentHashMap.KeySetView nettyResultIds = - ConcurrentHashMap.newKeySet(); - - private final ConcurrentHashMap.KeySetView - nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); - - /** - * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. - * @param resultPartitionManager A trivial {@link ResultPartitionManager}. - * @param resultPartitionFactory Factory class to create {@link RemoteShuffleResultPartition}. - * @param inputGateFactory Factory class to create {@link RemoteShuffleInputGate}. - * @param shuffleEnvironmentWrapper Wrapper class to create {@link NettyShuffleEnvironment}. - */ - public RemoteShuffleEnvironment( - NetworkBufferPool networkBufferPool, - ResultPartitionManager resultPartitionManager, - RemoteShuffleResultPartitionFactory resultPartitionFactory, - RemoteShuffleInputGateFactory inputGateFactory, - CelebornConf conf, - NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper) { - - super(networkBufferPool, resultPartitionManager, conf); - this.resultPartitionFactory = resultPartitionFactory; - this.inputGateFactory = inputGateFactory; - this.shuffleEnvironmentWrapper = shuffleEnvironmentWrapper; - } - - @Override - public ResultPartitionWriter createResultPartitionWriterInternal( - ShuffleIOOwnerContext ownerContext, - int index, - ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf) { - if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() - instanceof RemoteShuffleDescriptor) { - return resultPartitionFactory.create( - ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); - } else { - nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); - nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); - return shuffleEnvironmentWrapper - .nettyResultPartitionFactory() - .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); - } - } - - @Override - IndexedInputGate createInputGateInternal( - ShuffleIOOwnerContext ownerContext, - PartitionProducerStateProvider producerStateProvider, - int gateIndex, - InputGateDeploymentDescriptor igdd) { - return nettyResultIds.contains(igdd.getConsumedResultId()) - ? shuffleEnvironmentWrapper - .nettyInputGateFactory() - .create( - ownerContext, - gateIndex, - igdd, - producerStateProvider, - new InputChannelMetrics( - ownerContext.getInputGroup(), ownerContext.getParentGroup())) - : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); - } - - public void releasePartitionsLocally(Collection partitionIds) { - List resultPartitionIds = - partitionIds.stream() - .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) - .collect(Collectors.toList()); - if (!resultPartitionIds.isEmpty()) { - shuffleEnvironmentWrapper - .nettyShuffleEnvironment() - .releasePartitionsLocally(resultPartitionIds); - } - } - - @VisibleForTesting - RemoteShuffleResultPartitionFactory getResultPartitionFactory() { - return resultPartitionFactory; - } -} diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java index 6f34177c17f..37d7ce0dc19 100644 --- a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java +++ b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.flink.core.memory.MemorySegment; @@ -35,16 +33,13 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.SubpartitionIndexRange; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; @@ -53,16 +48,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.common.CelebornConf; /** A {@link IndexedInputGate} which ingest data from remote shuffle workers. */ -public class RemoteShuffleInputGate extends IndexedInputGate { - - private final RemoteShuffleInputGateDelegation inputGateDelegation; +public class RemoteShuffleInputGate extends AbstractRemoteShuffleInputGate { public RemoteShuffleInputGate( CelebornConf celebornConf, @@ -72,106 +64,14 @@ public RemoteShuffleInputGate( SupplierWithException bufferPoolFactory, BufferDecompressor bufferDecompressor, int numConcurrentReading) { - - inputGateDelegation = - new RemoteShuffleInputGateDelegation( - celebornConf, - taskName, - gateIndex, - gateDescriptor, - bufferPoolFactory, - bufferDecompressor, - numConcurrentReading, - availabilityHelper, - gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), - gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); - } - - /** Setup gate and build network connections. */ - @Override - public void setup() throws IOException { - inputGateDelegation.setup(); - } - - /** Index of the gate of the corresponding computing task. */ - @Override - public int getGateIndex() { - return inputGateDelegation.getGateIndex(); - } - - /** Get number of input channels. A channel is a data flow from one shuffle worker. */ - @Override - public int getNumberOfInputChannels() { - return inputGateDelegation.getBufferReaders().size(); - } - - /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ - @Override - public boolean isFinished() { - return inputGateDelegation.isFinished(); - } - - @Override - public Optional getNext() { - throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); - } - - /** Poll a received {@link BufferOrEvent}. */ - @Override - public Optional pollNext() throws IOException { - return inputGateDelegation.pollNext(); - } - - /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */ - @Override - public void close() throws Exception { - inputGateDelegation.close(); - } - - /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */ - @Override - public List getChannelInfos() { - return inputGateDelegation.getChannelsInfo(); - } - - @Override - public void requestPartitions() { - // do-nothing - } - - @Override - public void checkpointStarted(CheckpointBarrier barrier) { - // do-nothing. - } - - @Override - public void checkpointStopped(long cancelledCheckpointId) { - // do-nothing. - } - - @Override - public void triggerDebloating() { - // do-nothing. - } - - @Override - public List getUnfinishedChannels() { - return Collections.emptyList(); - } - - @Override - public EndOfDataStatus hasReceivedEndOfData() { - if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { - return EndOfDataStatus.NOT_END_OF_DATA; - } else { - // Keep compatibility with streaming mode. - return EndOfDataStatus.DRAINED; - } - } - - @Override - public void finishReadRecoveredState() { - // do-nothing. + super( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading); } @Override @@ -179,33 +79,6 @@ public InputChannel getChannel(int channelIndex) { return new FakedRemoteInputChannel(channelIndex); } - @Override - public void sendTaskEvent(TaskEvent event) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void resumeConsumption(InputChannelInfo channelInfo) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} - - @Override - public CompletableFuture getStateConsumedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return String.format( - "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", - inputGateDelegation.getTaskName(), - inputGateDelegation.getGateIndex(), - inputGateDelegation.getGateDescriptor().toString()); - } - /** Accommodation for the incompleteness of Flink pluggable shuffle service. */ private class FakedRemoteInputChannel extends RemoteInputChannel { FakedRemoteInputChannel(int channelIndex) { diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java index 5aa057ae121..cc33a64644f 100644 --- a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java +++ b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java @@ -17,33 +17,20 @@ package org.apache.celeborn.plugin.flink; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; - import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.network.api.EndOfData; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.*; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; import org.apache.celeborn.plugin.flink.buffer.DataBuffer; -import org.apache.celeborn.plugin.flink.utils.BufferUtils; -import org.apache.celeborn.plugin.flink.utils.Utils; /** * A {@link ResultPartition} which appends records and events to {@link DataBuffer} and after the @@ -51,11 +38,7 @@ * remote shuffle service in subpartition index order sequentially. Large records that can not be * appended to an empty {@link DataBuffer} will be spilled directly. */ -public class RemoteShuffleResultPartition extends ResultPartition { - - private final RemoteShuffleResultPartitionDelegation delegation; - - private final SupplierWithException bufferPoolFactory; +public class RemoteShuffleResultPartition extends AbstractRemoteShuffleResultPartition { public RemoteShuffleResultPartition( String owningTaskName, @@ -69,7 +52,6 @@ public RemoteShuffleResultPartition( @Nullable BufferCompressor bufferCompressor, SupplierWithException bufferPoolFactory, RemoteShuffleOutputGate outputGate) { - super( owningTaskName, partitionIndex, @@ -77,164 +59,15 @@ public RemoteShuffleResultPartition( partitionType, numSubpartitions, numTargetKeyGroups, + networkBufferSize, partitionManager, bufferCompressor, - bufferPoolFactory); - - delegation = - new RemoteShuffleResultPartitionDelegation( - networkBufferSize, - outputGate, - (bufferWithChannel, isBroadcast) -> updateStatistics(bufferWithChannel, isBroadcast), - numSubpartitions); - this.bufferPoolFactory = bufferPoolFactory; - } - - @Override - public void setup() throws IOException { - // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. - // The reason is that this partition will be registered to the partition manager during - // `super.setup()`. - // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), - // Flink does not trigger the resource releasing over TM. Therefore, the partition object is - // leaked. - // So we copy the logic of `setup` but don't register partition to partition manager. - checkState( - this.bufferPool == null, - "Bug in result partition setup logic: Already registered buffer pool."); - this.bufferPool = checkNotNull(bufferPoolFactory.get()); - // this is an empty method, but still call it in case of we implement it in the future. - setupInternal(); - BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); - delegation.setup( - bufferPool, - bufferCompressor, - buffer -> canBeCompressed(buffer), - () -> checkInProduceState()); - } - - @Override - protected void setupInternal() { - // do not need to implement - } - - @Override - public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { - delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); - } - - @Override - public void broadcastRecord(ByteBuffer record) throws IOException { - delegation.broadcast(record, DataType.DATA_BUFFER); - } - - @Override - public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { - Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); - try { - ByteBuffer serializedEvent = buffer.getNioBufferReadable(); - delegation.broadcast(serializedEvent, buffer.getDataType()); - } finally { - buffer.recycleBuffer(); - } - } - - @Override - public void alignedBarrierTimeout(long l) {} - - @Override - public void abortCheckpoint(long l, CheckpointException e) {} - - @Override - public void finish() throws IOException { - Utils.checkState(!isReleased(), "Result partition is already released."); - broadcastEvent(EndOfPartitionEvent.INSTANCE, false); - delegation.finish(); - super.finish(); - } - - @Override - public synchronized void close() { - delegation.close(() -> super.close()); - } - - @Override - protected void releaseInternal() { - // no-op - } - - @Override - public void flushAll() { - delegation.flushAll(); - } - - @Override - public void flush(int subpartitionIndex) { - flushAll(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public int getNumberOfQueuedBuffers() { - return 0; - } - - @Override - public long getSizeOfQueuedBuffersUnsafe() { - return 0; - } - - @Override - public int getNumberOfQueuedBuffers(int targetSubpartition) { - return 0; - } - - @Override - public ResultSubpartitionView createSubpartitionView( - int index, BufferAvailabilityListener availabilityListener) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void notifyEndOfData(StopMode mode) throws IOException { - if (!delegation.isEndOfDataNotified()) { - broadcastEvent(new EndOfData(mode), false); - delegation.setEndOfDataNotified(true); - } - } - - @Override - public CompletableFuture getAllDataProcessedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return "ResultPartition " - + partitionId.toString() - + " [" - + partitionType - + ", " - + numSubpartitions - + " subpartitions, shuffle-descriptor: " - + delegation.getOutputGate().getShuffleDesc() - + "]"; - } - - @VisibleForTesting - public RemoteShuffleResultPartitionDelegation getDelegation() { - return delegation; + bufferPoolFactory, + outputGate); } - public void updateStatistics(BufferWithSubpartition bufferWithChannel, boolean isBroadcast) { - numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); - long readableBytes = - (long) bufferWithChannel.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + public void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes) { numBytesProduced.inc(readableBytes); - numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); } } diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java index 7247795d3a0..a1b6688a784 100644 --- a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java +++ b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java @@ -17,30 +17,14 @@ package org.apache.celeborn.plugin.flink; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; -import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMasterContext; -import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; -public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory - implements ShuffleServiceFactory { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory = - new NettyShuffleServiceFactory(); - - @Override - public ShuffleMaster createShuffleMaster( - ShuffleMasterContext shuffleMasterContext) { - return new RemoteShuffleMaster( - shuffleMasterContext, new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); - } +public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory { @Override public ShuffleEnvironment createShuffleEnvironment( diff --git a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java b/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java deleted file mode 100644 index 3476b8fff0c..00000000000 --- a/client-flink/flink-1.16/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public class SimpleResultPartitionAdapter implements ResultPartitionAdapter { - @Override - public boolean isBlockingResultPartition(ResultPartitionType partitionType) { - return partitionType.isBlockingOrBlockingPersistentResultPartition(); - } -} diff --git a/client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java b/client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java index 236fde78a55..1567b4bb7ec 100644 --- a/client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java +++ b/client-flink/flink-1.16/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java @@ -321,7 +321,6 @@ public void onFatalError(Throwable throwable) { System.exit(-1); } }, - new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); return remoteShuffleMaster; diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java deleted file mode 100644 index 844fd276a99..00000000000 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; -import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.ShuffleEnvironment; -import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; - -import org.apache.celeborn.common.CelebornConf; -import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; - -/** - * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing - * shuffle environment on flink TM side. - */ -public class RemoteShuffleEnvironment extends AbstractRemoteShuffleEnvironment - implements ShuffleEnvironment { - - /** Factory class to create {@link RemoteShuffleResultPartition}. */ - private final RemoteShuffleResultPartitionFactory resultPartitionFactory; - - private final RemoteShuffleInputGateFactory inputGateFactory; - - private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; - - private final ConcurrentHashMap.KeySetView nettyResultIds = - ConcurrentHashMap.newKeySet(); - - private final ConcurrentHashMap.KeySetView - nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); - - /** - * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. - * @param resultPartitionManager A trivial {@link ResultPartitionManager}. - * @param resultPartitionFactory Factory class to create {@link RemoteShuffleResultPartition}. - * @param inputGateFactory Factory class to create {@link RemoteShuffleInputGate}. - * @param shuffleEnvironmentWrapper Wrapper class to create {@link NettyShuffleEnvironment}. - */ - public RemoteShuffleEnvironment( - NetworkBufferPool networkBufferPool, - ResultPartitionManager resultPartitionManager, - RemoteShuffleResultPartitionFactory resultPartitionFactory, - RemoteShuffleInputGateFactory inputGateFactory, - CelebornConf conf, - NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper) { - - super(networkBufferPool, resultPartitionManager, conf); - this.resultPartitionFactory = resultPartitionFactory; - this.inputGateFactory = inputGateFactory; - this.shuffleEnvironmentWrapper = shuffleEnvironmentWrapper; - } - - @Override - public ResultPartitionWriter createResultPartitionWriterInternal( - ShuffleIOOwnerContext ownerContext, - int index, - ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf) { - if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() - instanceof RemoteShuffleDescriptor) { - return resultPartitionFactory.create( - ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); - } else { - nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); - nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); - return shuffleEnvironmentWrapper - .nettyResultPartitionFactory() - .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); - } - } - - @Override - IndexedInputGate createInputGateInternal( - ShuffleIOOwnerContext ownerContext, - PartitionProducerStateProvider producerStateProvider, - int gateIndex, - InputGateDeploymentDescriptor igdd) { - return nettyResultIds.contains(igdd.getConsumedResultId()) - ? shuffleEnvironmentWrapper - .nettyInputGateFactory() - .create( - ownerContext, - gateIndex, - igdd, - producerStateProvider, - new InputChannelMetrics( - ownerContext.getInputGroup(), ownerContext.getParentGroup())) - : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); - } - - public void releasePartitionsLocally(Collection partitionIds) { - List resultPartitionIds = - partitionIds.stream() - .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) - .collect(Collectors.toList()); - if (!resultPartitionIds.isEmpty()) { - shuffleEnvironmentWrapper - .nettyShuffleEnvironment() - .releasePartitionsLocally(resultPartitionIds); - } - } - - @VisibleForTesting - RemoteShuffleResultPartitionFactory getResultPartitionFactory() { - return resultPartitionFactory; - } -} diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java index 20545696b49..1e153a02419 100644 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java +++ b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.flink.core.memory.MemorySegment; @@ -34,17 +32,14 @@ import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.IndexRange; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; @@ -53,16 +48,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.common.CelebornConf; /** A {@link IndexedInputGate} which ingest data from remote shuffle workers. */ -public class RemoteShuffleInputGate extends IndexedInputGate { - - private final RemoteShuffleInputGateDelegation inputGateDelegation; +public class RemoteShuffleInputGate extends AbstractRemoteShuffleInputGate { public RemoteShuffleInputGate( CelebornConf celebornConf, @@ -72,106 +64,14 @@ public RemoteShuffleInputGate( SupplierWithException bufferPoolFactory, BufferDecompressor bufferDecompressor, int numConcurrentReading) { - - inputGateDelegation = - new RemoteShuffleInputGateDelegation( - celebornConf, - taskName, - gateIndex, - gateDescriptor, - bufferPoolFactory, - bufferDecompressor, - numConcurrentReading, - availabilityHelper, - gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), - gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); - } - - /** Setup gate and build network connections. */ - @Override - public void setup() throws IOException { - inputGateDelegation.setup(); - } - - /** Index of the gate of the corresponding computing task. */ - @Override - public int getGateIndex() { - return inputGateDelegation.getGateIndex(); - } - - /** Get number of input channels. A channel is a data flow from one shuffle worker. */ - @Override - public int getNumberOfInputChannels() { - return inputGateDelegation.getBufferReaders().size(); - } - - /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ - @Override - public boolean isFinished() { - return inputGateDelegation.isFinished(); - } - - @Override - public Optional getNext() { - throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); - } - - /** Poll a received {@link BufferOrEvent}. */ - @Override - public Optional pollNext() throws IOException { - return inputGateDelegation.pollNext(); - } - - /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */ - @Override - public void close() throws Exception { - inputGateDelegation.close(); - } - - /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */ - @Override - public List getChannelInfos() { - return inputGateDelegation.getChannelsInfo(); - } - - @Override - public void requestPartitions() { - // do-nothing - } - - @Override - public void checkpointStarted(CheckpointBarrier barrier) { - // do-nothing. - } - - @Override - public void checkpointStopped(long cancelledCheckpointId) { - // do-nothing. - } - - @Override - public void triggerDebloating() { - // do-nothing. - } - - @Override - public List getUnfinishedChannels() { - return Collections.emptyList(); - } - - @Override - public EndOfDataStatus hasReceivedEndOfData() { - if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { - return EndOfDataStatus.NOT_END_OF_DATA; - } else { - // Keep compatibility with streaming mode. - return EndOfDataStatus.DRAINED; - } - } - - @Override - public void finishReadRecoveredState() { - // do-nothing. + super( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading); } @Override @@ -179,33 +79,6 @@ public InputChannel getChannel(int channelIndex) { return new FakedRemoteInputChannel(channelIndex); } - @Override - public void sendTaskEvent(TaskEvent event) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void resumeConsumption(InputChannelInfo channelInfo) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} - - @Override - public CompletableFuture getStateConsumedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return String.format( - "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", - inputGateDelegation.getTaskName(), - inputGateDelegation.getGateIndex(), - inputGateDelegation.getGateDescriptor().toString()); - } - /** Accommodation for the incompleteness of Flink pluggable shuffle service. */ private class FakedRemoteInputChannel extends RemoteInputChannel { FakedRemoteInputChannel(int channelIndex) { diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java index 8aaa013c547..256acb828ed 100644 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java +++ b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java @@ -17,33 +17,20 @@ package org.apache.celeborn.plugin.flink; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; - import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.network.api.EndOfData; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.*; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; import org.apache.celeborn.plugin.flink.buffer.DataBuffer; -import org.apache.celeborn.plugin.flink.utils.BufferUtils; -import org.apache.celeborn.plugin.flink.utils.Utils; /** * A {@link ResultPartition} which appends records and events to {@link DataBuffer} and after the @@ -51,11 +38,7 @@ * remote shuffle service in subpartition index order sequentially. Large records that can not be * appended to an empty {@link DataBuffer} will be spilled directly. */ -public class RemoteShuffleResultPartition extends ResultPartition { - - private final RemoteShuffleResultPartitionDelegation delegation; - - private final SupplierWithException bufferPoolFactory; +public class RemoteShuffleResultPartition extends AbstractRemoteShuffleResultPartition { public RemoteShuffleResultPartition( String owningTaskName, @@ -69,7 +52,6 @@ public RemoteShuffleResultPartition( @Nullable BufferCompressor bufferCompressor, SupplierWithException bufferPoolFactory, RemoteShuffleOutputGate outputGate) { - super( owningTaskName, partitionIndex, @@ -77,162 +59,19 @@ public RemoteShuffleResultPartition( partitionType, numSubpartitions, numTargetKeyGroups, + networkBufferSize, partitionManager, bufferCompressor, - bufferPoolFactory); - - delegation = - new RemoteShuffleResultPartitionDelegation( - networkBufferSize, outputGate, this::updateStatistics, numSubpartitions); - this.bufferPoolFactory = bufferPoolFactory; - } - - @Override - public void setup() throws IOException { - // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. - // The reason is that this partition will be registered to the partition manager during - // `super.setup()`. - // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), - // Flink does not trigger the resource releasing over TM. Therefore, the partition object is - // leaked. - // So we copy the logic of `setup` but don't register partition to partition manager. - checkState( - this.bufferPool == null, - "Bug in result partition setup logic: Already registered buffer pool."); - this.bufferPool = checkNotNull(bufferPoolFactory.get()); - // this is an empty method, but still call it in case of we implement it in the future. - setupInternal(); - BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); - delegation.setup( - bufferPool, bufferCompressor, this::canBeCompressed, this::checkInProduceState); - } - - @Override - protected void setupInternal() { - // do not need to implement - } - - @Override - public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { - delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); - } - - @Override - public void broadcastRecord(ByteBuffer record) throws IOException { - delegation.broadcast(record, DataType.DATA_BUFFER); - } - - @Override - public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { - Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); - try { - ByteBuffer serializedEvent = buffer.getNioBufferReadable(); - delegation.broadcast(serializedEvent, buffer.getDataType()); - } finally { - buffer.recycleBuffer(); - } - } - - @Override - public void alignedBarrierTimeout(long l) {} - - @Override - public void abortCheckpoint(long l, CheckpointException e) {} - - @Override - public void finish() throws IOException { - Utils.checkState(!isReleased(), "Result partition is already released."); - broadcastEvent(EndOfPartitionEvent.INSTANCE, false); - delegation.finish(); - super.finish(); - } - - @Override - public synchronized void close() { - delegation.close(super::close); - } - - @Override - protected void releaseInternal() { - // no-op - } - - @Override - public void flushAll() { - delegation.flushAll(); - } - - @Override - public void flush(int subpartitionIndex) { - flushAll(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public int getNumberOfQueuedBuffers() { - return 0; - } - - @Override - public long getSizeOfQueuedBuffersUnsafe() { - return 0; - } - - @Override - public int getNumberOfQueuedBuffers(int targetSubpartition) { - return 0; - } - - @Override - public ResultSubpartitionView createSubpartitionView( - int index, BufferAvailabilityListener availabilityListener) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void notifyEndOfData(StopMode mode) throws IOException { - if (!delegation.isEndOfDataNotified()) { - broadcastEvent(new EndOfData(mode), false); - delegation.setEndOfDataNotified(true); - } - } - - @Override - public CompletableFuture getAllDataProcessedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return "ResultPartition " - + partitionId.toString() - + " [" - + partitionType - + ", " - + numSubpartitions - + " subpartitions, shuffle-descriptor: " - + delegation.getOutputGate().getShuffleDesc() - + "]"; - } - - @VisibleForTesting - public RemoteShuffleResultPartitionDelegation getDelegation() { - return delegation; + bufferPoolFactory, + outputGate); } - public void updateStatistics(BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast) { - numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); - long readableBytes = - (long) bufferWithSubpartition.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + public void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes) { if (isBroadcast) { resultPartitionBytes.incAll(readableBytes); } else { resultPartitionBytes.inc(bufferWithSubpartition.getSubpartitionIndex(), readableBytes); } - numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); } } diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java index 7247795d3a0..a1b6688a784 100644 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java +++ b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java @@ -17,30 +17,14 @@ package org.apache.celeborn.plugin.flink; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; -import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMasterContext; -import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; -public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory - implements ShuffleServiceFactory { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory = - new NettyShuffleServiceFactory(); - - @Override - public ShuffleMaster createShuffleMaster( - ShuffleMasterContext shuffleMasterContext) { - return new RemoteShuffleMaster( - shuffleMasterContext, new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); - } +public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory { @Override public ShuffleEnvironment createShuffleEnvironment( diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java deleted file mode 100644 index 3476b8fff0c..00000000000 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public class SimpleResultPartitionAdapter implements ResultPartitionAdapter { - @Override - public boolean isBlockingResultPartition(ResultPartitionType partitionType) { - return partitionType.isBlockingOrBlockingPersistentResultPartition(); - } -} diff --git a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java b/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java deleted file mode 100644 index 3706feeffb5..00000000000 --- a/client-flink/flink-1.17/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.celeborn.plugin.flink.netty; - -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; -import org.apache.flink.runtime.io.network.partition.ResultPartitionFactory; -import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; -import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; - -import org.apache.celeborn.reflect.DynFields; - -/** - * The wrapper of {@link NettyShuffleEnvironment} to generate {@link ResultPartitionFactory} and - * {@link SingleInputGateFactory}. - */ -public class NettyShuffleEnvironmentWrapper { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory; - private final ShuffleEnvironmentContext shuffleEnvironmentContext; - - private volatile NettyShuffleEnvironment nettyShuffleEnvironment; - private volatile ResultPartitionFactory nettyResultPartitionFactory; - private volatile SingleInputGateFactory nettyInputGateFactory; - - private static final DynFields.UnboundField - RESULT_PARTITION_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "resultPartitionFactory") - .defaultAlwaysNull() - .build(); - - private static final DynFields.UnboundField INPUT_GATE_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "singleInputGateFactory") - .defaultAlwaysNull() - .build(); - - public NettyShuffleEnvironmentWrapper( - NettyShuffleServiceFactory nettyShuffleServiceFactory, - ShuffleEnvironmentContext shuffleEnvironmentContext) { - this.nettyShuffleServiceFactory = nettyShuffleServiceFactory; - this.shuffleEnvironmentContext = shuffleEnvironmentContext; - } - - public NettyShuffleEnvironment nettyShuffleEnvironment() { - if (nettyShuffleEnvironment == null) { - synchronized (this) { - if (nettyShuffleEnvironment == null) { - nettyShuffleEnvironment = - nettyShuffleServiceFactory.createShuffleEnvironment(shuffleEnvironmentContext); - } - } - } - return nettyShuffleEnvironment; - } - - public ResultPartitionFactory nettyResultPartitionFactory() { - if (nettyResultPartitionFactory == null) { - synchronized (this) { - if (nettyResultPartitionFactory == null) { - nettyResultPartitionFactory = - RESULT_PARTITION_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyResultPartitionFactory; - } - - public SingleInputGateFactory nettyInputGateFactory() { - if (nettyInputGateFactory == null) { - synchronized (this) { - if (nettyInputGateFactory == null) { - nettyInputGateFactory = INPUT_GATE_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyInputGateFactory; - } -} diff --git a/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java b/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java index 3ff73db5a42..046af49387e 100644 --- a/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java +++ b/client-flink/flink-1.17/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java @@ -333,7 +333,6 @@ public void onFatalError(Throwable throwable) { System.exit(-1); } }, - new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); return remoteShuffleMaster; diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java deleted file mode 100644 index 844fd276a99..00000000000 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; -import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.ShuffleEnvironment; -import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; - -import org.apache.celeborn.common.CelebornConf; -import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; - -/** - * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing - * shuffle environment on flink TM side. - */ -public class RemoteShuffleEnvironment extends AbstractRemoteShuffleEnvironment - implements ShuffleEnvironment { - - /** Factory class to create {@link RemoteShuffleResultPartition}. */ - private final RemoteShuffleResultPartitionFactory resultPartitionFactory; - - private final RemoteShuffleInputGateFactory inputGateFactory; - - private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; - - private final ConcurrentHashMap.KeySetView nettyResultIds = - ConcurrentHashMap.newKeySet(); - - private final ConcurrentHashMap.KeySetView - nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); - - /** - * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. - * @param resultPartitionManager A trivial {@link ResultPartitionManager}. - * @param resultPartitionFactory Factory class to create {@link RemoteShuffleResultPartition}. - * @param inputGateFactory Factory class to create {@link RemoteShuffleInputGate}. - * @param shuffleEnvironmentWrapper Wrapper class to create {@link NettyShuffleEnvironment}. - */ - public RemoteShuffleEnvironment( - NetworkBufferPool networkBufferPool, - ResultPartitionManager resultPartitionManager, - RemoteShuffleResultPartitionFactory resultPartitionFactory, - RemoteShuffleInputGateFactory inputGateFactory, - CelebornConf conf, - NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper) { - - super(networkBufferPool, resultPartitionManager, conf); - this.resultPartitionFactory = resultPartitionFactory; - this.inputGateFactory = inputGateFactory; - this.shuffleEnvironmentWrapper = shuffleEnvironmentWrapper; - } - - @Override - public ResultPartitionWriter createResultPartitionWriterInternal( - ShuffleIOOwnerContext ownerContext, - int index, - ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf) { - if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() - instanceof RemoteShuffleDescriptor) { - return resultPartitionFactory.create( - ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); - } else { - nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); - nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); - return shuffleEnvironmentWrapper - .nettyResultPartitionFactory() - .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); - } - } - - @Override - IndexedInputGate createInputGateInternal( - ShuffleIOOwnerContext ownerContext, - PartitionProducerStateProvider producerStateProvider, - int gateIndex, - InputGateDeploymentDescriptor igdd) { - return nettyResultIds.contains(igdd.getConsumedResultId()) - ? shuffleEnvironmentWrapper - .nettyInputGateFactory() - .create( - ownerContext, - gateIndex, - igdd, - producerStateProvider, - new InputChannelMetrics( - ownerContext.getInputGroup(), ownerContext.getParentGroup())) - : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); - } - - public void releasePartitionsLocally(Collection partitionIds) { - List resultPartitionIds = - partitionIds.stream() - .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) - .collect(Collectors.toList()); - if (!resultPartitionIds.isEmpty()) { - shuffleEnvironmentWrapper - .nettyShuffleEnvironment() - .releasePartitionsLocally(resultPartitionIds); - } - } - - @VisibleForTesting - RemoteShuffleResultPartitionFactory getResultPartitionFactory() { - return resultPartitionFactory; - } -} diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java index d04f7182ff5..862741a502b 100644 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java +++ b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.flink.core.memory.MemorySegment; @@ -34,17 +32,14 @@ import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.IndexRange; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; @@ -53,16 +48,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.common.CelebornConf; /** A {@link IndexedInputGate} which ingest data from remote shuffle workers. */ -public class RemoteShuffleInputGate extends IndexedInputGate { - - private final RemoteShuffleInputGateDelegation inputGateDelegation; +public class RemoteShuffleInputGate extends AbstractRemoteShuffleInputGate { public RemoteShuffleInputGate( CelebornConf celebornConf, @@ -72,106 +64,14 @@ public RemoteShuffleInputGate( SupplierWithException bufferPoolFactory, BufferDecompressor bufferDecompressor, int numConcurrentReading) { - - inputGateDelegation = - new RemoteShuffleInputGateDelegation( - celebornConf, - taskName, - gateIndex, - gateDescriptor, - bufferPoolFactory, - bufferDecompressor, - numConcurrentReading, - availabilityHelper, - gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), - gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); - } - - /** Setup gate and build network connections. */ - @Override - public void setup() throws IOException { - inputGateDelegation.setup(); - } - - /** Index of the gate of the corresponding computing task. */ - @Override - public int getGateIndex() { - return inputGateDelegation.getGateIndex(); - } - - /** Get number of input channels. A channel is a data flow from one shuffle worker. */ - @Override - public int getNumberOfInputChannels() { - return inputGateDelegation.getBufferReaders().size(); - } - - /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ - @Override - public boolean isFinished() { - return inputGateDelegation.isFinished(); - } - - @Override - public Optional getNext() { - throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); - } - - /** Poll a received {@link BufferOrEvent}. */ - @Override - public Optional pollNext() throws IOException { - return inputGateDelegation.pollNext(); - } - - /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */ - @Override - public void close() throws Exception { - inputGateDelegation.close(); - } - - /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */ - @Override - public List getChannelInfos() { - return inputGateDelegation.getChannelsInfo(); - } - - @Override - public void requestPartitions() { - // do-nothing - } - - @Override - public void checkpointStarted(CheckpointBarrier barrier) { - // do-nothing. - } - - @Override - public void checkpointStopped(long cancelledCheckpointId) { - // do-nothing. - } - - @Override - public void triggerDebloating() { - // do-nothing. - } - - @Override - public List getUnfinishedChannels() { - return Collections.emptyList(); - } - - @Override - public EndOfDataStatus hasReceivedEndOfData() { - if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { - return EndOfDataStatus.NOT_END_OF_DATA; - } else { - // Keep compatibility with streaming mode. - return EndOfDataStatus.DRAINED; - } - } - - @Override - public void finishReadRecoveredState() { - // do-nothing. + super( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading); } @Override @@ -179,33 +79,6 @@ public InputChannel getChannel(int channelIndex) { return new FakedRemoteInputChannel(channelIndex); } - @Override - public void sendTaskEvent(TaskEvent event) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void resumeConsumption(InputChannelInfo channelInfo) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} - - @Override - public CompletableFuture getStateConsumedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return String.format( - "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", - inputGateDelegation.getTaskName(), - inputGateDelegation.getGateIndex(), - inputGateDelegation.getGateDescriptor().toString()); - } - /** Accommodation for the incompleteness of Flink pluggable shuffle service. */ private class FakedRemoteInputChannel extends RemoteInputChannel { FakedRemoteInputChannel(int channelIndex) { diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java index 8aaa013c547..256acb828ed 100644 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java +++ b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java @@ -17,33 +17,20 @@ package org.apache.celeborn.plugin.flink; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; - import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.network.api.EndOfData; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.*; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; import org.apache.celeborn.plugin.flink.buffer.DataBuffer; -import org.apache.celeborn.plugin.flink.utils.BufferUtils; -import org.apache.celeborn.plugin.flink.utils.Utils; /** * A {@link ResultPartition} which appends records and events to {@link DataBuffer} and after the @@ -51,11 +38,7 @@ * remote shuffle service in subpartition index order sequentially. Large records that can not be * appended to an empty {@link DataBuffer} will be spilled directly. */ -public class RemoteShuffleResultPartition extends ResultPartition { - - private final RemoteShuffleResultPartitionDelegation delegation; - - private final SupplierWithException bufferPoolFactory; +public class RemoteShuffleResultPartition extends AbstractRemoteShuffleResultPartition { public RemoteShuffleResultPartition( String owningTaskName, @@ -69,7 +52,6 @@ public RemoteShuffleResultPartition( @Nullable BufferCompressor bufferCompressor, SupplierWithException bufferPoolFactory, RemoteShuffleOutputGate outputGate) { - super( owningTaskName, partitionIndex, @@ -77,162 +59,19 @@ public RemoteShuffleResultPartition( partitionType, numSubpartitions, numTargetKeyGroups, + networkBufferSize, partitionManager, bufferCompressor, - bufferPoolFactory); - - delegation = - new RemoteShuffleResultPartitionDelegation( - networkBufferSize, outputGate, this::updateStatistics, numSubpartitions); - this.bufferPoolFactory = bufferPoolFactory; - } - - @Override - public void setup() throws IOException { - // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. - // The reason is that this partition will be registered to the partition manager during - // `super.setup()`. - // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), - // Flink does not trigger the resource releasing over TM. Therefore, the partition object is - // leaked. - // So we copy the logic of `setup` but don't register partition to partition manager. - checkState( - this.bufferPool == null, - "Bug in result partition setup logic: Already registered buffer pool."); - this.bufferPool = checkNotNull(bufferPoolFactory.get()); - // this is an empty method, but still call it in case of we implement it in the future. - setupInternal(); - BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); - delegation.setup( - bufferPool, bufferCompressor, this::canBeCompressed, this::checkInProduceState); - } - - @Override - protected void setupInternal() { - // do not need to implement - } - - @Override - public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { - delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); - } - - @Override - public void broadcastRecord(ByteBuffer record) throws IOException { - delegation.broadcast(record, DataType.DATA_BUFFER); - } - - @Override - public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { - Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); - try { - ByteBuffer serializedEvent = buffer.getNioBufferReadable(); - delegation.broadcast(serializedEvent, buffer.getDataType()); - } finally { - buffer.recycleBuffer(); - } - } - - @Override - public void alignedBarrierTimeout(long l) {} - - @Override - public void abortCheckpoint(long l, CheckpointException e) {} - - @Override - public void finish() throws IOException { - Utils.checkState(!isReleased(), "Result partition is already released."); - broadcastEvent(EndOfPartitionEvent.INSTANCE, false); - delegation.finish(); - super.finish(); - } - - @Override - public synchronized void close() { - delegation.close(super::close); - } - - @Override - protected void releaseInternal() { - // no-op - } - - @Override - public void flushAll() { - delegation.flushAll(); - } - - @Override - public void flush(int subpartitionIndex) { - flushAll(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public int getNumberOfQueuedBuffers() { - return 0; - } - - @Override - public long getSizeOfQueuedBuffersUnsafe() { - return 0; - } - - @Override - public int getNumberOfQueuedBuffers(int targetSubpartition) { - return 0; - } - - @Override - public ResultSubpartitionView createSubpartitionView( - int index, BufferAvailabilityListener availabilityListener) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void notifyEndOfData(StopMode mode) throws IOException { - if (!delegation.isEndOfDataNotified()) { - broadcastEvent(new EndOfData(mode), false); - delegation.setEndOfDataNotified(true); - } - } - - @Override - public CompletableFuture getAllDataProcessedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return "ResultPartition " - + partitionId.toString() - + " [" - + partitionType - + ", " - + numSubpartitions - + " subpartitions, shuffle-descriptor: " - + delegation.getOutputGate().getShuffleDesc() - + "]"; - } - - @VisibleForTesting - public RemoteShuffleResultPartitionDelegation getDelegation() { - return delegation; + bufferPoolFactory, + outputGate); } - public void updateStatistics(BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast) { - numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); - long readableBytes = - (long) bufferWithSubpartition.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + public void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes) { if (isBroadcast) { resultPartitionBytes.incAll(readableBytes); } else { resultPartitionBytes.inc(bufferWithSubpartition.getSubpartitionIndex(), readableBytes); } - numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); } } diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java index 7247795d3a0..a1b6688a784 100644 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java +++ b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java @@ -17,30 +17,14 @@ package org.apache.celeborn.plugin.flink; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; -import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMasterContext; -import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; -public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory - implements ShuffleServiceFactory { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory = - new NettyShuffleServiceFactory(); - - @Override - public ShuffleMaster createShuffleMaster( - ShuffleMasterContext shuffleMasterContext) { - return new RemoteShuffleMaster( - shuffleMasterContext, new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); - } +public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory { @Override public ShuffleEnvironment createShuffleEnvironment( diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java deleted file mode 100644 index 3476b8fff0c..00000000000 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public class SimpleResultPartitionAdapter implements ResultPartitionAdapter { - @Override - public boolean isBlockingResultPartition(ResultPartitionType partitionType) { - return partitionType.isBlockingOrBlockingPersistentResultPartition(); - } -} diff --git a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java b/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java deleted file mode 100644 index 3706feeffb5..00000000000 --- a/client-flink/flink-1.18/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.celeborn.plugin.flink.netty; - -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; -import org.apache.flink.runtime.io.network.partition.ResultPartitionFactory; -import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; -import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; - -import org.apache.celeborn.reflect.DynFields; - -/** - * The wrapper of {@link NettyShuffleEnvironment} to generate {@link ResultPartitionFactory} and - * {@link SingleInputGateFactory}. - */ -public class NettyShuffleEnvironmentWrapper { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory; - private final ShuffleEnvironmentContext shuffleEnvironmentContext; - - private volatile NettyShuffleEnvironment nettyShuffleEnvironment; - private volatile ResultPartitionFactory nettyResultPartitionFactory; - private volatile SingleInputGateFactory nettyInputGateFactory; - - private static final DynFields.UnboundField - RESULT_PARTITION_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "resultPartitionFactory") - .defaultAlwaysNull() - .build(); - - private static final DynFields.UnboundField INPUT_GATE_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "singleInputGateFactory") - .defaultAlwaysNull() - .build(); - - public NettyShuffleEnvironmentWrapper( - NettyShuffleServiceFactory nettyShuffleServiceFactory, - ShuffleEnvironmentContext shuffleEnvironmentContext) { - this.nettyShuffleServiceFactory = nettyShuffleServiceFactory; - this.shuffleEnvironmentContext = shuffleEnvironmentContext; - } - - public NettyShuffleEnvironment nettyShuffleEnvironment() { - if (nettyShuffleEnvironment == null) { - synchronized (this) { - if (nettyShuffleEnvironment == null) { - nettyShuffleEnvironment = - nettyShuffleServiceFactory.createShuffleEnvironment(shuffleEnvironmentContext); - } - } - } - return nettyShuffleEnvironment; - } - - public ResultPartitionFactory nettyResultPartitionFactory() { - if (nettyResultPartitionFactory == null) { - synchronized (this) { - if (nettyResultPartitionFactory == null) { - nettyResultPartitionFactory = - RESULT_PARTITION_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyResultPartitionFactory; - } - - public SingleInputGateFactory nettyInputGateFactory() { - if (nettyInputGateFactory == null) { - synchronized (this) { - if (nettyInputGateFactory == null) { - nettyInputGateFactory = INPUT_GATE_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyInputGateFactory; - } -} diff --git a/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java b/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java index 3ff73db5a42..046af49387e 100644 --- a/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java +++ b/client-flink/flink-1.18/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java @@ -333,7 +333,6 @@ public void onFatalError(Throwable throwable) { System.exit(-1); } }, - new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); return remoteShuffleMaster; diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java deleted file mode 100644 index 844fd276a99..00000000000 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; -import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.ShuffleEnvironment; -import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; - -import org.apache.celeborn.common.CelebornConf; -import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; - -/** - * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing - * shuffle environment on flink TM side. - */ -public class RemoteShuffleEnvironment extends AbstractRemoteShuffleEnvironment - implements ShuffleEnvironment { - - /** Factory class to create {@link RemoteShuffleResultPartition}. */ - private final RemoteShuffleResultPartitionFactory resultPartitionFactory; - - private final RemoteShuffleInputGateFactory inputGateFactory; - - private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; - - private final ConcurrentHashMap.KeySetView nettyResultIds = - ConcurrentHashMap.newKeySet(); - - private final ConcurrentHashMap.KeySetView - nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); - - /** - * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. - * @param resultPartitionManager A trivial {@link ResultPartitionManager}. - * @param resultPartitionFactory Factory class to create {@link RemoteShuffleResultPartition}. - * @param inputGateFactory Factory class to create {@link RemoteShuffleInputGate}. - * @param shuffleEnvironmentWrapper Wrapper class to create {@link NettyShuffleEnvironment}. - */ - public RemoteShuffleEnvironment( - NetworkBufferPool networkBufferPool, - ResultPartitionManager resultPartitionManager, - RemoteShuffleResultPartitionFactory resultPartitionFactory, - RemoteShuffleInputGateFactory inputGateFactory, - CelebornConf conf, - NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper) { - - super(networkBufferPool, resultPartitionManager, conf); - this.resultPartitionFactory = resultPartitionFactory; - this.inputGateFactory = inputGateFactory; - this.shuffleEnvironmentWrapper = shuffleEnvironmentWrapper; - } - - @Override - public ResultPartitionWriter createResultPartitionWriterInternal( - ShuffleIOOwnerContext ownerContext, - int index, - ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf) { - if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() - instanceof RemoteShuffleDescriptor) { - return resultPartitionFactory.create( - ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); - } else { - nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); - nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); - return shuffleEnvironmentWrapper - .nettyResultPartitionFactory() - .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); - } - } - - @Override - IndexedInputGate createInputGateInternal( - ShuffleIOOwnerContext ownerContext, - PartitionProducerStateProvider producerStateProvider, - int gateIndex, - InputGateDeploymentDescriptor igdd) { - return nettyResultIds.contains(igdd.getConsumedResultId()) - ? shuffleEnvironmentWrapper - .nettyInputGateFactory() - .create( - ownerContext, - gateIndex, - igdd, - producerStateProvider, - new InputChannelMetrics( - ownerContext.getInputGroup(), ownerContext.getParentGroup())) - : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); - } - - public void releasePartitionsLocally(Collection partitionIds) { - List resultPartitionIds = - partitionIds.stream() - .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) - .collect(Collectors.toList()); - if (!resultPartitionIds.isEmpty()) { - shuffleEnvironmentWrapper - .nettyShuffleEnvironment() - .releasePartitionsLocally(resultPartitionIds); - } - } - - @VisibleForTesting - RemoteShuffleResultPartitionFactory getResultPartitionFactory() { - return resultPartitionFactory; - } -} diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java index 7617aaf44b1..615565f5879 100644 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java +++ b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.flink.core.memory.MemorySegment; @@ -34,18 +32,15 @@ import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.IndexRange; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionIndexSet; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; @@ -54,16 +49,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.common.CelebornConf; /** A {@link IndexedInputGate} which ingest data from remote shuffle workers. */ -public class RemoteShuffleInputGate extends IndexedInputGate { - - private final RemoteShuffleInputGateDelegation inputGateDelegation; +public class RemoteShuffleInputGate extends AbstractRemoteShuffleInputGate { public RemoteShuffleInputGate( CelebornConf celebornConf, @@ -73,106 +65,14 @@ public RemoteShuffleInputGate( SupplierWithException bufferPoolFactory, BufferDecompressor bufferDecompressor, int numConcurrentReading) { - - inputGateDelegation = - new RemoteShuffleInputGateDelegation( - celebornConf, - taskName, - gateIndex, - gateDescriptor, - bufferPoolFactory, - bufferDecompressor, - numConcurrentReading, - availabilityHelper, - gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), - gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); - } - - /** Setup gate and build network connections. */ - @Override - public void setup() throws IOException { - inputGateDelegation.setup(); - } - - /** Index of the gate of the corresponding computing task. */ - @Override - public int getGateIndex() { - return inputGateDelegation.getGateIndex(); - } - - /** Get number of input channels. A channel is a data flow from one shuffle worker. */ - @Override - public int getNumberOfInputChannels() { - return inputGateDelegation.getBufferReaders().size(); - } - - /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ - @Override - public boolean isFinished() { - return inputGateDelegation.isFinished(); - } - - @Override - public Optional getNext() { - throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); - } - - /** Poll a received {@link BufferOrEvent}. */ - @Override - public Optional pollNext() throws IOException { - return inputGateDelegation.pollNext(); - } - - /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */ - @Override - public void close() throws Exception { - inputGateDelegation.close(); - } - - /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */ - @Override - public List getChannelInfos() { - return inputGateDelegation.getChannelsInfo(); - } - - @Override - public void requestPartitions() { - // do-nothing - } - - @Override - public void checkpointStarted(CheckpointBarrier barrier) { - // do-nothing. - } - - @Override - public void checkpointStopped(long cancelledCheckpointId) { - // do-nothing. - } - - @Override - public void triggerDebloating() { - // do-nothing. - } - - @Override - public List getUnfinishedChannels() { - return Collections.emptyList(); - } - - @Override - public EndOfDataStatus hasReceivedEndOfData() { - if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { - return EndOfDataStatus.NOT_END_OF_DATA; - } else { - // Keep compatibility with streaming mode. - return EndOfDataStatus.DRAINED; - } - } - - @Override - public void finishReadRecoveredState() { - // do-nothing. + super( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading); } @Override @@ -180,33 +80,6 @@ public InputChannel getChannel(int channelIndex) { return new FakedRemoteInputChannel(channelIndex); } - @Override - public void sendTaskEvent(TaskEvent event) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void resumeConsumption(InputChannelInfo channelInfo) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} - - @Override - public CompletableFuture getStateConsumedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return String.format( - "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", - inputGateDelegation.getTaskName(), - inputGateDelegation.getGateIndex(), - inputGateDelegation.getGateDescriptor().toString()); - } - /** Accommodation for the incompleteness of Flink pluggable shuffle service. */ private class FakedRemoteInputChannel extends RemoteInputChannel { FakedRemoteInputChannel(int channelIndex) { diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java index 8aaa013c547..256acb828ed 100644 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java +++ b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java @@ -17,33 +17,20 @@ package org.apache.celeborn.plugin.flink; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; - import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.network.api.EndOfData; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.*; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; import org.apache.celeborn.plugin.flink.buffer.DataBuffer; -import org.apache.celeborn.plugin.flink.utils.BufferUtils; -import org.apache.celeborn.plugin.flink.utils.Utils; /** * A {@link ResultPartition} which appends records and events to {@link DataBuffer} and after the @@ -51,11 +38,7 @@ * remote shuffle service in subpartition index order sequentially. Large records that can not be * appended to an empty {@link DataBuffer} will be spilled directly. */ -public class RemoteShuffleResultPartition extends ResultPartition { - - private final RemoteShuffleResultPartitionDelegation delegation; - - private final SupplierWithException bufferPoolFactory; +public class RemoteShuffleResultPartition extends AbstractRemoteShuffleResultPartition { public RemoteShuffleResultPartition( String owningTaskName, @@ -69,7 +52,6 @@ public RemoteShuffleResultPartition( @Nullable BufferCompressor bufferCompressor, SupplierWithException bufferPoolFactory, RemoteShuffleOutputGate outputGate) { - super( owningTaskName, partitionIndex, @@ -77,162 +59,19 @@ public RemoteShuffleResultPartition( partitionType, numSubpartitions, numTargetKeyGroups, + networkBufferSize, partitionManager, bufferCompressor, - bufferPoolFactory); - - delegation = - new RemoteShuffleResultPartitionDelegation( - networkBufferSize, outputGate, this::updateStatistics, numSubpartitions); - this.bufferPoolFactory = bufferPoolFactory; - } - - @Override - public void setup() throws IOException { - // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. - // The reason is that this partition will be registered to the partition manager during - // `super.setup()`. - // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), - // Flink does not trigger the resource releasing over TM. Therefore, the partition object is - // leaked. - // So we copy the logic of `setup` but don't register partition to partition manager. - checkState( - this.bufferPool == null, - "Bug in result partition setup logic: Already registered buffer pool."); - this.bufferPool = checkNotNull(bufferPoolFactory.get()); - // this is an empty method, but still call it in case of we implement it in the future. - setupInternal(); - BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); - delegation.setup( - bufferPool, bufferCompressor, this::canBeCompressed, this::checkInProduceState); - } - - @Override - protected void setupInternal() { - // do not need to implement - } - - @Override - public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { - delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); - } - - @Override - public void broadcastRecord(ByteBuffer record) throws IOException { - delegation.broadcast(record, DataType.DATA_BUFFER); - } - - @Override - public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { - Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); - try { - ByteBuffer serializedEvent = buffer.getNioBufferReadable(); - delegation.broadcast(serializedEvent, buffer.getDataType()); - } finally { - buffer.recycleBuffer(); - } - } - - @Override - public void alignedBarrierTimeout(long l) {} - - @Override - public void abortCheckpoint(long l, CheckpointException e) {} - - @Override - public void finish() throws IOException { - Utils.checkState(!isReleased(), "Result partition is already released."); - broadcastEvent(EndOfPartitionEvent.INSTANCE, false); - delegation.finish(); - super.finish(); - } - - @Override - public synchronized void close() { - delegation.close(super::close); - } - - @Override - protected void releaseInternal() { - // no-op - } - - @Override - public void flushAll() { - delegation.flushAll(); - } - - @Override - public void flush(int subpartitionIndex) { - flushAll(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public int getNumberOfQueuedBuffers() { - return 0; - } - - @Override - public long getSizeOfQueuedBuffersUnsafe() { - return 0; - } - - @Override - public int getNumberOfQueuedBuffers(int targetSubpartition) { - return 0; - } - - @Override - public ResultSubpartitionView createSubpartitionView( - int index, BufferAvailabilityListener availabilityListener) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void notifyEndOfData(StopMode mode) throws IOException { - if (!delegation.isEndOfDataNotified()) { - broadcastEvent(new EndOfData(mode), false); - delegation.setEndOfDataNotified(true); - } - } - - @Override - public CompletableFuture getAllDataProcessedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return "ResultPartition " - + partitionId.toString() - + " [" - + partitionType - + ", " - + numSubpartitions - + " subpartitions, shuffle-descriptor: " - + delegation.getOutputGate().getShuffleDesc() - + "]"; - } - - @VisibleForTesting - public RemoteShuffleResultPartitionDelegation getDelegation() { - return delegation; + bufferPoolFactory, + outputGate); } - public void updateStatistics(BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast) { - numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); - long readableBytes = - (long) bufferWithSubpartition.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + public void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes) { if (isBroadcast) { resultPartitionBytes.incAll(readableBytes); } else { resultPartitionBytes.inc(bufferWithSubpartition.getSubpartitionIndex(), readableBytes); } - numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); } } diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java index 7247795d3a0..a1b6688a784 100644 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java +++ b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java @@ -17,30 +17,14 @@ package org.apache.celeborn.plugin.flink; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; -import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMasterContext; -import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; -public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory - implements ShuffleServiceFactory { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory = - new NettyShuffleServiceFactory(); - - @Override - public ShuffleMaster createShuffleMaster( - ShuffleMasterContext shuffleMasterContext) { - return new RemoteShuffleMaster( - shuffleMasterContext, new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); - } +public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory { @Override public ShuffleEnvironment createShuffleEnvironment( diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java deleted file mode 100644 index 3476b8fff0c..00000000000 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public class SimpleResultPartitionAdapter implements ResultPartitionAdapter { - @Override - public boolean isBlockingResultPartition(ResultPartitionType partitionType) { - return partitionType.isBlockingOrBlockingPersistentResultPartition(); - } -} diff --git a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java b/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java deleted file mode 100644 index 3706feeffb5..00000000000 --- a/client-flink/flink-1.19/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.celeborn.plugin.flink.netty; - -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; -import org.apache.flink.runtime.io.network.partition.ResultPartitionFactory; -import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; -import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; - -import org.apache.celeborn.reflect.DynFields; - -/** - * The wrapper of {@link NettyShuffleEnvironment} to generate {@link ResultPartitionFactory} and - * {@link SingleInputGateFactory}. - */ -public class NettyShuffleEnvironmentWrapper { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory; - private final ShuffleEnvironmentContext shuffleEnvironmentContext; - - private volatile NettyShuffleEnvironment nettyShuffleEnvironment; - private volatile ResultPartitionFactory nettyResultPartitionFactory; - private volatile SingleInputGateFactory nettyInputGateFactory; - - private static final DynFields.UnboundField - RESULT_PARTITION_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "resultPartitionFactory") - .defaultAlwaysNull() - .build(); - - private static final DynFields.UnboundField INPUT_GATE_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "singleInputGateFactory") - .defaultAlwaysNull() - .build(); - - public NettyShuffleEnvironmentWrapper( - NettyShuffleServiceFactory nettyShuffleServiceFactory, - ShuffleEnvironmentContext shuffleEnvironmentContext) { - this.nettyShuffleServiceFactory = nettyShuffleServiceFactory; - this.shuffleEnvironmentContext = shuffleEnvironmentContext; - } - - public NettyShuffleEnvironment nettyShuffleEnvironment() { - if (nettyShuffleEnvironment == null) { - synchronized (this) { - if (nettyShuffleEnvironment == null) { - nettyShuffleEnvironment = - nettyShuffleServiceFactory.createShuffleEnvironment(shuffleEnvironmentContext); - } - } - } - return nettyShuffleEnvironment; - } - - public ResultPartitionFactory nettyResultPartitionFactory() { - if (nettyResultPartitionFactory == null) { - synchronized (this) { - if (nettyResultPartitionFactory == null) { - nettyResultPartitionFactory = - RESULT_PARTITION_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyResultPartitionFactory; - } - - public SingleInputGateFactory nettyInputGateFactory() { - if (nettyInputGateFactory == null) { - synchronized (this) { - if (nettyInputGateFactory == null) { - nettyInputGateFactory = INPUT_GATE_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyInputGateFactory; - } -} diff --git a/client-flink/flink-1.19/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java b/client-flink/flink-1.19/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java index b1a3fadb941..b702f6063dc 100644 --- a/client-flink/flink-1.19/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java +++ b/client-flink/flink-1.19/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java @@ -333,7 +333,6 @@ public void onFatalError(Throwable throwable) { System.exit(-1); } }, - new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); return remoteShuffleMaster; diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java deleted file mode 100644 index f07e26645f0..00000000000 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleEnvironment.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; -import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.ShuffleEnvironment; -import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; - -import org.apache.celeborn.common.CelebornConf; -import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; - -/** - * The implementation of {@link ShuffleEnvironment} based on the remote shuffle service, providing - * shuffle environment on flink TM side. - */ -public class RemoteShuffleEnvironment extends AbstractRemoteShuffleEnvironment - implements ShuffleEnvironment { - - /** Factory class to create {@link RemoteShuffleResultPartition}. */ - private final RemoteShuffleResultPartitionFactory resultPartitionFactory; - - private final RemoteShuffleInputGateFactory inputGateFactory; - - private final NettyShuffleEnvironmentWrapper shuffleEnvironmentWrapper; - - private final ConcurrentHashMap.KeySetView nettyResultIds = - ConcurrentHashMap.newKeySet(); - - private final ConcurrentHashMap.KeySetView - nettyResultPartitionIds = ConcurrentHashMap.newKeySet(); - - /** - * @param networkBufferPool Network buffer pool for shuffle read and shuffle write. - * @param resultPartitionManager A trivial {@link ResultPartitionManager}. - * @param resultPartitionFactory Factory class to create {@link RemoteShuffleResultPartition}. - * @param inputGateFactory Factory class to create {@link RemoteShuffleInputGate}. - * @param nettyShuffleEnvironmentWrapper Wrapper class to create {@link NettyShuffleEnvironment}. - */ - public RemoteShuffleEnvironment( - NetworkBufferPool networkBufferPool, - ResultPartitionManager resultPartitionManager, - RemoteShuffleResultPartitionFactory resultPartitionFactory, - RemoteShuffleInputGateFactory inputGateFactory, - CelebornConf conf, - NettyShuffleEnvironmentWrapper nettyShuffleEnvironmentWrapper) { - - super(networkBufferPool, resultPartitionManager, conf); - this.resultPartitionFactory = resultPartitionFactory; - this.inputGateFactory = inputGateFactory; - this.shuffleEnvironmentWrapper = nettyShuffleEnvironmentWrapper; - } - - @Override - public ResultPartitionWriter createResultPartitionWriterInternal( - ShuffleIOOwnerContext ownerContext, - int index, - ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, - CelebornConf conf) { - if (resultPartitionDeploymentDescriptor.getShuffleDescriptor() - instanceof RemoteShuffleDescriptor) { - return resultPartitionFactory.create( - ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor, conf); - } else { - nettyResultIds.add(resultPartitionDeploymentDescriptor.getResultId()); - nettyResultPartitionIds.add(resultPartitionDeploymentDescriptor.getPartitionId()); - return shuffleEnvironmentWrapper - .nettyResultPartitionFactory() - .create(ownerContext.getOwnerName(), index, resultPartitionDeploymentDescriptor); - } - } - - @Override - IndexedInputGate createInputGateInternal( - ShuffleIOOwnerContext ownerContext, - PartitionProducerStateProvider producerStateProvider, - int gateIndex, - InputGateDeploymentDescriptor igdd) { - return nettyResultIds.contains(igdd.getConsumedResultId()) - ? shuffleEnvironmentWrapper - .nettyInputGateFactory() - .create( - ownerContext, - gateIndex, - igdd, - producerStateProvider, - new InputChannelMetrics( - ownerContext.getInputGroup(), ownerContext.getParentGroup())) - : inputGateFactory.create(ownerContext.getOwnerName(), gateIndex, igdd); - } - - public void releasePartitionsLocally(Collection partitionIds) { - List resultPartitionIds = - partitionIds.stream() - .filter(partitionId -> nettyResultPartitionIds.contains(partitionId.getPartitionId())) - .collect(Collectors.toList()); - if (!resultPartitionIds.isEmpty()) { - shuffleEnvironmentWrapper - .nettyShuffleEnvironment() - .releasePartitionsLocally(resultPartitionIds); - } - } - - @VisibleForTesting - RemoteShuffleResultPartitionFactory getResultPartitionFactory() { - return resultPartitionFactory; - } -} diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java index 7617aaf44b1..615565f5879 100644 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java +++ b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleInputGate.java @@ -20,9 +20,7 @@ import java.io.IOException; import java.net.InetAddress; import java.util.Collection; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.flink.core.memory.MemorySegment; @@ -34,18 +32,15 @@ import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.IndexRange; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionIndexSet; -import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; @@ -54,16 +49,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.common.CelebornConf; /** A {@link IndexedInputGate} which ingest data from remote shuffle workers. */ -public class RemoteShuffleInputGate extends IndexedInputGate { - - private final RemoteShuffleInputGateDelegation inputGateDelegation; +public class RemoteShuffleInputGate extends AbstractRemoteShuffleInputGate { public RemoteShuffleInputGate( CelebornConf celebornConf, @@ -73,106 +65,14 @@ public RemoteShuffleInputGate( SupplierWithException bufferPoolFactory, BufferDecompressor bufferDecompressor, int numConcurrentReading) { - - inputGateDelegation = - new RemoteShuffleInputGateDelegation( - celebornConf, - taskName, - gateIndex, - gateDescriptor, - bufferPoolFactory, - bufferDecompressor, - numConcurrentReading, - availabilityHelper, - gateDescriptor.getConsumedSubpartitionIndexRange().getStartIndex(), - gateDescriptor.getConsumedSubpartitionIndexRange().getEndIndex()); - } - - /** Setup gate and build network connections. */ - @Override - public void setup() throws IOException { - inputGateDelegation.setup(); - } - - /** Index of the gate of the corresponding computing task. */ - @Override - public int getGateIndex() { - return inputGateDelegation.getGateIndex(); - } - - /** Get number of input channels. A channel is a data flow from one shuffle worker. */ - @Override - public int getNumberOfInputChannels() { - return inputGateDelegation.getBufferReaders().size(); - } - - /** Whether reading is finished -- all channels are finished and cached buffers are drained. */ - @Override - public boolean isFinished() { - return inputGateDelegation.isFinished(); - } - - @Override - public Optional getNext() { - throw new UnsupportedOperationException("Not implemented (DataSet API is not supported)."); - } - - /** Poll a received {@link BufferOrEvent}. */ - @Override - public Optional pollNext() throws IOException { - return inputGateDelegation.pollNext(); - } - - /** Close all reading channels inside this {@link RemoteShuffleInputGate}. */ - @Override - public void close() throws Exception { - inputGateDelegation.close(); - } - - /** Get {@link InputChannelInfo}s of this {@link RemoteShuffleInputGate}. */ - @Override - public List getChannelInfos() { - return inputGateDelegation.getChannelsInfo(); - } - - @Override - public void requestPartitions() { - // do-nothing - } - - @Override - public void checkpointStarted(CheckpointBarrier barrier) { - // do-nothing. - } - - @Override - public void checkpointStopped(long cancelledCheckpointId) { - // do-nothing. - } - - @Override - public void triggerDebloating() { - // do-nothing. - } - - @Override - public List getUnfinishedChannels() { - return Collections.emptyList(); - } - - @Override - public EndOfDataStatus hasReceivedEndOfData() { - if (inputGateDelegation.getPendingEndOfDataEvents() > 0) { - return EndOfDataStatus.NOT_END_OF_DATA; - } else { - // Keep compatibility with streaming mode. - return EndOfDataStatus.DRAINED; - } - } - - @Override - public void finishReadRecoveredState() { - // do-nothing. + super( + celebornConf, + taskName, + gateIndex, + gateDescriptor, + bufferPoolFactory, + bufferDecompressor, + numConcurrentReading); } @Override @@ -180,33 +80,6 @@ public InputChannel getChannel(int channelIndex) { return new FakedRemoteInputChannel(channelIndex); } - @Override - public void sendTaskEvent(TaskEvent event) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void resumeConsumption(InputChannelInfo channelInfo) { - throw new FlinkRuntimeException("Method should not be called."); - } - - @Override - public void acknowledgeAllRecordsProcessed(InputChannelInfo inputChannelInfo) {} - - @Override - public CompletableFuture getStateConsumedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return String.format( - "ReadGate [owning task: %s, gate index: %d, descriptor: %s]", - inputGateDelegation.getTaskName(), - inputGateDelegation.getGateIndex(), - inputGateDelegation.getGateDescriptor().toString()); - } - /** Accommodation for the incompleteness of Flink pluggable shuffle service. */ private class FakedRemoteInputChannel extends RemoteInputChannel { FakedRemoteInputChannel(int channelIndex) { diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java index 8aaa013c547..256acb828ed 100644 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java +++ b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleResultPartition.java @@ -17,33 +17,20 @@ package org.apache.celeborn.plugin.flink; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkNotNull; -import static org.apache.celeborn.plugin.flink.utils.Utils.checkState; - import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.event.AbstractEvent; -import org.apache.flink.runtime.io.network.api.EndOfData; -import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; -import org.apache.flink.runtime.io.network.api.StopMode; -import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.*; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.util.function.SupplierWithException; import org.apache.celeborn.plugin.flink.buffer.BufferWithSubpartition; import org.apache.celeborn.plugin.flink.buffer.DataBuffer; -import org.apache.celeborn.plugin.flink.utils.BufferUtils; -import org.apache.celeborn.plugin.flink.utils.Utils; /** * A {@link ResultPartition} which appends records and events to {@link DataBuffer} and after the @@ -51,11 +38,7 @@ * remote shuffle service in subpartition index order sequentially. Large records that can not be * appended to an empty {@link DataBuffer} will be spilled directly. */ -public class RemoteShuffleResultPartition extends ResultPartition { - - private final RemoteShuffleResultPartitionDelegation delegation; - - private final SupplierWithException bufferPoolFactory; +public class RemoteShuffleResultPartition extends AbstractRemoteShuffleResultPartition { public RemoteShuffleResultPartition( String owningTaskName, @@ -69,7 +52,6 @@ public RemoteShuffleResultPartition( @Nullable BufferCompressor bufferCompressor, SupplierWithException bufferPoolFactory, RemoteShuffleOutputGate outputGate) { - super( owningTaskName, partitionIndex, @@ -77,162 +59,19 @@ public RemoteShuffleResultPartition( partitionType, numSubpartitions, numTargetKeyGroups, + networkBufferSize, partitionManager, bufferCompressor, - bufferPoolFactory); - - delegation = - new RemoteShuffleResultPartitionDelegation( - networkBufferSize, outputGate, this::updateStatistics, numSubpartitions); - this.bufferPoolFactory = bufferPoolFactory; - } - - @Override - public void setup() throws IOException { - // We can't call the `setup` method of the base class, otherwise it will cause a partition leak. - // The reason is that this partition will be registered to the partition manager during - // `super.setup()`. - // Since this is a cluster/remote partition(i.e. resources are not stored on the Flink TM), - // Flink does not trigger the resource releasing over TM. Therefore, the partition object is - // leaked. - // So we copy the logic of `setup` but don't register partition to partition manager. - checkState( - this.bufferPool == null, - "Bug in result partition setup logic: Already registered buffer pool."); - this.bufferPool = checkNotNull(bufferPoolFactory.get()); - // this is an empty method, but still call it in case of we implement it in the future. - setupInternal(); - BufferUtils.reserveNumRequiredBuffers(bufferPool, 1); - delegation.setup( - bufferPool, bufferCompressor, this::canBeCompressed, this::checkInProduceState); - } - - @Override - protected void setupInternal() { - // do not need to implement - } - - @Override - public void emitRecord(ByteBuffer record, int targetSubpartition) throws IOException { - delegation.emit(record, targetSubpartition, DataType.DATA_BUFFER, false); - } - - @Override - public void broadcastRecord(ByteBuffer record) throws IOException { - delegation.broadcast(record, DataType.DATA_BUFFER); - } - - @Override - public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent) throws IOException { - Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent); - try { - ByteBuffer serializedEvent = buffer.getNioBufferReadable(); - delegation.broadcast(serializedEvent, buffer.getDataType()); - } finally { - buffer.recycleBuffer(); - } - } - - @Override - public void alignedBarrierTimeout(long l) {} - - @Override - public void abortCheckpoint(long l, CheckpointException e) {} - - @Override - public void finish() throws IOException { - Utils.checkState(!isReleased(), "Result partition is already released."); - broadcastEvent(EndOfPartitionEvent.INSTANCE, false); - delegation.finish(); - super.finish(); - } - - @Override - public synchronized void close() { - delegation.close(super::close); - } - - @Override - protected void releaseInternal() { - // no-op - } - - @Override - public void flushAll() { - delegation.flushAll(); - } - - @Override - public void flush(int subpartitionIndex) { - flushAll(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public int getNumberOfQueuedBuffers() { - return 0; - } - - @Override - public long getSizeOfQueuedBuffersUnsafe() { - return 0; - } - - @Override - public int getNumberOfQueuedBuffers(int targetSubpartition) { - return 0; - } - - @Override - public ResultSubpartitionView createSubpartitionView( - int index, BufferAvailabilityListener availabilityListener) { - throw new UnsupportedOperationException("Not supported."); - } - - @Override - public void notifyEndOfData(StopMode mode) throws IOException { - if (!delegation.isEndOfDataNotified()) { - broadcastEvent(new EndOfData(mode), false); - delegation.setEndOfDataNotified(true); - } - } - - @Override - public CompletableFuture getAllDataProcessedFuture() { - return CompletableFuture.completedFuture(null); - } - - @Override - public String toString() { - return "ResultPartition " - + partitionId.toString() - + " [" - + partitionType - + ", " - + numSubpartitions - + " subpartitions, shuffle-descriptor: " - + delegation.getOutputGate().getShuffleDesc() - + "]"; - } - - @VisibleForTesting - public RemoteShuffleResultPartitionDelegation getDelegation() { - return delegation; + bufferPoolFactory, + outputGate); } - public void updateStatistics(BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast) { - numBuffersOut.inc(isBroadcast ? numSubpartitions : 1); - long readableBytes = - (long) bufferWithSubpartition.getBuffer().readableBytes() - BufferUtils.HEADER_LENGTH; + public void updateReadableBytes( + BufferWithSubpartition bufferWithSubpartition, boolean isBroadcast, long readableBytes) { if (isBroadcast) { resultPartitionBytes.incAll(readableBytes); } else { resultPartitionBytes.inc(bufferWithSubpartition.getSubpartitionIndex(), readableBytes); } - numBytesOut.inc(isBroadcast ? readableBytes * numSubpartitions : readableBytes); } } diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java index 7247795d3a0..a1b6688a784 100644 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java +++ b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/RemoteShuffleServiceFactory.java @@ -17,30 +17,14 @@ package org.apache.celeborn.plugin.flink; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate; -import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; -import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMasterContext; -import org.apache.flink.runtime.shuffle.ShuffleServiceFactory; import org.apache.celeborn.plugin.flink.netty.NettyShuffleEnvironmentWrapper; -public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory - implements ShuffleServiceFactory { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory = - new NettyShuffleServiceFactory(); - - @Override - public ShuffleMaster createShuffleMaster( - ShuffleMasterContext shuffleMasterContext) { - return new RemoteShuffleMaster( - shuffleMasterContext, new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); - } +public class RemoteShuffleServiceFactory extends AbstractRemoteShuffleServiceFactory { @Override public ShuffleEnvironment createShuffleEnvironment( diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java deleted file mode 100644 index 3476b8fff0c..00000000000 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/SimpleResultPartitionAdapter.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.celeborn.plugin.flink; - -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; - -public class SimpleResultPartitionAdapter implements ResultPartitionAdapter { - @Override - public boolean isBlockingResultPartition(ResultPartitionType partitionType) { - return partitionType.isBlockingOrBlockingPersistentResultPartition(); - } -} diff --git a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java b/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java deleted file mode 100644 index 3706feeffb5..00000000000 --- a/client-flink/flink-1.20/src/main/java/org/apache/celeborn/plugin/flink/netty/NettyShuffleEnvironmentWrapper.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.celeborn.plugin.flink.netty; - -import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; -import org.apache.flink.runtime.io.network.NettyShuffleServiceFactory; -import org.apache.flink.runtime.io.network.partition.ResultPartitionFactory; -import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateFactory; -import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; - -import org.apache.celeborn.reflect.DynFields; - -/** - * The wrapper of {@link NettyShuffleEnvironment} to generate {@link ResultPartitionFactory} and - * {@link SingleInputGateFactory}. - */ -public class NettyShuffleEnvironmentWrapper { - - private final NettyShuffleServiceFactory nettyShuffleServiceFactory; - private final ShuffleEnvironmentContext shuffleEnvironmentContext; - - private volatile NettyShuffleEnvironment nettyShuffleEnvironment; - private volatile ResultPartitionFactory nettyResultPartitionFactory; - private volatile SingleInputGateFactory nettyInputGateFactory; - - private static final DynFields.UnboundField - RESULT_PARTITION_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "resultPartitionFactory") - .defaultAlwaysNull() - .build(); - - private static final DynFields.UnboundField INPUT_GATE_FACTORY_FIELD = - DynFields.builder() - .hiddenImpl(NettyShuffleEnvironment.class, "singleInputGateFactory") - .defaultAlwaysNull() - .build(); - - public NettyShuffleEnvironmentWrapper( - NettyShuffleServiceFactory nettyShuffleServiceFactory, - ShuffleEnvironmentContext shuffleEnvironmentContext) { - this.nettyShuffleServiceFactory = nettyShuffleServiceFactory; - this.shuffleEnvironmentContext = shuffleEnvironmentContext; - } - - public NettyShuffleEnvironment nettyShuffleEnvironment() { - if (nettyShuffleEnvironment == null) { - synchronized (this) { - if (nettyShuffleEnvironment == null) { - nettyShuffleEnvironment = - nettyShuffleServiceFactory.createShuffleEnvironment(shuffleEnvironmentContext); - } - } - } - return nettyShuffleEnvironment; - } - - public ResultPartitionFactory nettyResultPartitionFactory() { - if (nettyResultPartitionFactory == null) { - synchronized (this) { - if (nettyResultPartitionFactory == null) { - nettyResultPartitionFactory = - RESULT_PARTITION_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyResultPartitionFactory; - } - - public SingleInputGateFactory nettyInputGateFactory() { - if (nettyInputGateFactory == null) { - synchronized (this) { - if (nettyInputGateFactory == null) { - nettyInputGateFactory = INPUT_GATE_FACTORY_FIELD.bind(nettyShuffleEnvironment()).get(); - } - } - } - return nettyInputGateFactory; - } -} diff --git a/client-flink/flink-1.20/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java b/client-flink/flink-1.20/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java index 03718df18e1..ddcb176bc41 100644 --- a/client-flink/flink-1.20/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java +++ b/client-flink/flink-1.20/src/test/java/org/apache/celeborn/plugin/flink/RemoteShuffleMasterSuiteJ.java @@ -336,7 +336,6 @@ public void onFatalError(Throwable throwable) { System.exit(-1); } }, - new SimpleResultPartitionAdapter(), nettyShuffleServiceFactory); return remoteShuffleMaster;