TanYuxin-tyx commented on code in PR #22728: URL: https://github.com/apache/flink/pull/22728#discussion_r1226172397
########## flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierProducerAgent.java: ########## @@ -0,0 +1,175 @@ +/* + * 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.flink.runtime.io.network.partition.hybrid.tiered.tier.memory; + +import org.apache.flink.runtime.io.network.api.EndOfSegmentEvent; +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.partition.hybrid.tiered.common.TieredStoragePartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageSubpartitionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionId; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyConnectionWriter; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.NettyServiceProducer; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry; +import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent; +import org.apache.flink.util.ExceptionUtils; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The memory tier implementation of {@link TierProducerAgent}. */ +public class MemoryTierProducerAgent implements TierProducerAgent, NettyServiceProducer { + + private final int segmentSizeBytes; + + private final int segmentSizeBuffers; + + private final TieredStorageMemoryManager memoryManager; + + private final boolean isBroadcastOnly; + + // Record the byte number currently written to each subpartition. + private final int[] numSubpartitionEmitBytes; + + /** + * Whether a subpartition has been registered to the netty service. The array index is + * corresponding to the subpartition id. + */ + private final boolean[] nettyServiceRegistered; + + private final MemoryTierSubpartitionProducerAgent[] subpartitionProducerAgents; + + public MemoryTierProducerAgent( + TieredStoragePartitionId partitionId, + int numSubpartitions, + int bufferSize, + int segmentSizeBytes, + boolean isBroadcastOnly, + TieredStorageMemoryManager memoryManager, + TieredStorageNettyService nettyService, + TieredStorageResourceRegistry resourceRegistry) { + checkArgument(segmentSizeBytes >= bufferSize, "One segment contains at least one buffer."); + + this.segmentSizeBytes = segmentSizeBytes; + this.segmentSizeBuffers = segmentSizeBytes / bufferSize; + this.isBroadcastOnly = isBroadcastOnly; + this.memoryManager = memoryManager; + this.numSubpartitionEmitBytes = new int[numSubpartitions]; + this.nettyServiceRegistered = new boolean[numSubpartitions]; + this.subpartitionProducerAgents = new MemoryTierSubpartitionProducerAgent[numSubpartitions]; + + Arrays.fill(numSubpartitionEmitBytes, 0); + nettyService.registerProducer(partitionId, this); + for (int subpartitionId = 0; subpartitionId < numSubpartitions; ++subpartitionId) { + subpartitionProducerAgents[subpartitionId] = + new MemoryTierSubpartitionProducerAgent(subpartitionId, nettyService); + } + resourceRegistry.registerResource(partitionId, this::releaseResources); + } + + @Override + public boolean tryStartNewSegment(TieredStorageSubpartitionId subpartitionId, int segmentId) { + if (isBroadcastOnly) { + return false; + } + boolean canStartNewSegment = + isSubpartitionRegistered(subpartitionId) + && (memoryManager.getMaxNonReclaimableBuffers(this) + - memoryManager.numOwnerRequestedBuffer(this)) + > segmentSizeBuffers; + if (canStartNewSegment) { + getSubpartitionProducerAgent(subpartitionId.getSubpartitionId()) + .addSegmentId(segmentId); + } + return canStartNewSegment; + } + + @Override + public boolean tryWrite(TieredStorageSubpartitionId subpartitionId, Buffer finishedBuffer) { + int subpartitionIndex = subpartitionId.getSubpartitionId(); + if (numSubpartitionEmitBytes[subpartitionIndex] != 0 + && numSubpartitionEmitBytes[subpartitionIndex] + finishedBuffer.readableBytes() + > segmentSizeBytes) { + appendEndOfSegmentEvent(subpartitionIndex); + numSubpartitionEmitBytes[subpartitionIndex] = 0; + return false; + } Review Comment: Rename this to `currentSubpartitionWriteBuffers`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
