一文搞懂 Flink Task 数据交互之数据写源码

一文搞懂 Flink Task 数据交互之数据写源码

  • 1. RecordWriterOutput
  • 2. RecordWriter
  • 3. 数据分区器ChannelSelector
  • 4. 数据输出模型ResultPartition
  • 5. 子模型ResultSubpartition
  • 6. 本地buffer池LocalBufferPool
  • 7. 获取buffer
  • 8. 将buffer添加到ResultSubpartition

Flink 重要源码目录点击我

Flink中的Task也有着类似于Map/Reduce的计算模型,Task之间的数据交换是相当重要的模块。本文是分析Flink Task数据写的过程,为了更好的了解在实时流计算过程中数据的生产、传输、消费的过程打个基础。对于批处理过程的数据传输也有意义

Flink实时流计算Task任务中有算子链OperatorChain,在算子链中,每个算子都持有一个Output接口, 上一个算子把计算处理完的数据通过Output接口把传递给下一个算子处理。那末尾的算子怎么办呢?当然是把处理完的数据写出去,末尾算子它持有的Output接口就是RecordWriterOutput。

1. RecordWriterOutput

public class RecordWriterOutput<OUT> implements OperatorChain.WatermarkGaugeExposingOutput<StreamRecord<OUT>> {

   private RecordWriter<SerializationDelegate<StreamElement>> recordWriter;

   private SerializationDelegate<StreamElement> serializationDelegate;

   private final StreamStatusProvider streamStatusProvider;

   private final OutputTag outputTag;

   private final WatermarkGauge watermarkGauge = new WatermarkGauge();
   
   @Override
   public void collect(StreamRecord<OUT> record) {
       if (this.outputTag != null) {
          // we are not responsible for emitting to the main output.
          return;
       }
    
       pushToRecordWriter(record);
    }
    
   private <X> void pushToRecordWriter(StreamRecord<X> record) {
       serializationDelegate.setInstance(record);
       try {
          // 最重要的就是这个RecordWriter了,collect()方法就是调用的RecordWriter.emit()方法
          recordWriter.emit(serializationDelegate);
       }
       catch (Exception e) {
          throw new RuntimeException(e.getMessage(), e);
       }
   }
}

2. RecordWriter

public abstract class RecordWriter<T extends IOReadableWritable> implements AvailabilityProvider {

   /** Default name for the output flush thread, if no name with a task reference is given. */
   @VisibleForTesting
   public static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher";

   private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class);

   // 最核心的成员了,也就是我们所熟知的ResultPartition。如果我们把任务模型当做map/reduce,那么每个map任务就拥有一个ResultPartition
   private final ResultPartitionWriter targetPartition;
   // 对应的是下游多少个任务,一般来说就是下游的总任务数,也就是reduce端的总任务数。除非数据的交换模式是POINTWISE,那可能只对应了一个任务
   protected final int numberOfChannels;
 
   protected final RecordSerializer<T> serializer;

   protected final Random rng = new XORShiftRandom();

   private Counter numBytesOut = new SimpleCounter();

   private Counter numBuffersOut = new SimpleCounter();

   protected Meter idleTimeMsPerSecond = new MeterView(new SimpleCounter());

   private final boolean flushAlways;

   /** The thread that periodically flushes the output, to give an upper latency bound. */
   @Nullable
   // 定时数据刷新器,是一个单独的线程。如果在上游数据产出较慢的情况下,确保数据可以及时发送到下游任务,不用等到写满buffer之后再发送。定时间隔为buffer的timeout
   private final OutputFlusher outputFlusher;
}

构造方法中设置了各个成员变量,构建了outputFlusher,outputFlusher是一个线程,outputFlusher的作用是定期去触发执行flushAll()方法,定时间隔为buffer的timeout,将会触发ResultPartition.flushAll()方法,将ResultPartition中的数据发给下游任务

// 构造函数
RecordWriter(ResultPartitionWriter writer, long timeout, String taskName) {
   this.targetPartition = writer;
   this.numberOfChannels = writer.getNumberOfSubpartitions();

   this.serializer = new SpanningRecordSerializer<T>();

   checkArgument(timeout >= -1);
   this.flushAlways = (timeout == 0);
   if (timeout == -1 || timeout == 0) {
      outputFlusher = null;
   } else {
      String threadName = taskName == null ?
         DEFAULT_OUTPUT_FLUSH_THREAD_NAME :
         DEFAULT_OUTPUT_FLUSH_THREAD_NAME + " for " + taskName;

      outputFlusher = new OutputFlusher(threadName, timeout);
      outputFlusher.start();
   }
}
private class OutputFlusher extends Thread {

   private final long timeout;
   private volatile boolean running = true;

   ...

   @Override
   public void run() {
      try {
         while (running) {
            try {
               Thread.sleep(timeout);
            } ...
            // any errors here should let the thread come to a halt and be
            // recognized by the writer
            flushAll();
         }
      } catch (Throwable t) {
         notifyFlusherException(t);
      }
   }
}

下面来看关键的emit()方法
1、首先选择数据对应的下游通道,也就是对应的下游哪个任务,有多种分区器实现
2、将数据进行序列化,获取对应通道的bufferBuilder,将数据拷贝到bufferBuilder里面
3、如果buffer写满了,标记buffer为finish,从ResultPartition里申请新的buffer,申请新buffer时如果没有空闲的buffer就会被阻塞,这时写操作就被阻塞了
4、申请到了buffer之后,就可以继续使用这个buffer来写数据了。同时ResultPartition会把这个buffer添加到ResultSubpartition的buffers数据列表中,这个稍后分析

//RecordWriter
public void emit(T record) throws IOException, InterruptedException {
   checkErroneous();
   //通过分区器选择数据对应的下游通道,也即对应的下游哪个任务
   emit(record, channelSelector.selectChannel(record));
}

private void emit(T record, int targetChannel) throws IOException, InterruptedException {
   //将数据进行序列化
   serializer.serializeRecord(record);

   if (copyFromSerializerToTargetChannel(targetChannel)) {
      serializer.prune();
   }
}

private boolean copyFromSerializerToTargetChannel(int targetChannel) throws IOException, InterruptedException {
   // We should reset the initial position of the intermediate serialization buffer before
   // copying, so the serialization results can be copied to multiple target buffers.
   serializer.reset();

   boolean pruneTriggered = false;
   //获取对应通道的buffer
   BufferBuilder bufferBuilder = getBufferBuilder(targetChannel);
   //将数据拷贝到buffer里
   SerializationResult result = serializer.copyToBufferBuilder(bufferBuilder);
   while (result.isFullBuffer()) {
       //buffer写满了,标记为finish
      numBytesOut.inc(bufferBuilder.finish());
      numBuffersOut.inc();

      // If this was a full record, we are done. Not breaking out of the loop at this point
      // will lead to another buffer request before breaking out (that would not be a
      // problem per se, but it can lead to stalls in the pipeline).
      if (result.isFullRecord()) {
         pruneTriggered = true;
         bufferBuilders[targetChannel] = Optional.empty();
         break;
      }
      //如果buffer写满了,但是刚才的记录只拷贝了一半,就申请新的buffer继续写
      bufferBuilder = requestNewBufferBuilder(targetChannel);
      result = serializer.copyToBufferBuilder(bufferBuilder);
   }
   checkState(!serializer.hasSerializedData(), "All data should be written at once");

   if (flushAlways) {
      targetPartition.flush(targetChannel);
   }
   return pruneTriggered;
}

3. 数据分区器ChannelSelector

下面看channelSelector的通道选择,是如何将数据分到下游不同的任务
首先我们看最常见的基于key的分组分区器KeyGroupStreamPartitioner,例如代码里使用keyBy()算子就会使用这种分区器

//KeyGroupStreamPartitioner类
public int selectChannel(SerializationDelegate<StreamRecord<T>> record) {
   K key;
   try {
      // 获取 key 
      key = keySelector.getKey(record.getInstance().getValue());
   } catch (Exception e) {
      throw new RuntimeException("Could not extract key from " + record.getInstance().getValue(), e);
   }
   return KeyGroupRangeAssignment.assignKeyToParallelOperator(key, maxParallelism, numberOfChannels);
}

//KeyGroupRangeAssignment类
public static int assignKeyToParallelOperator(Object key, int maxParallelism, int parallelism) {
   return computeOperatorIndexForKeyGroup(maxParallelism, parallelism, assignToKeyGroup(key, maxParallelism));
}

//计算key所属的下游任务(下游通道)
public static int computeOperatorIndexForKeyGroup(int maxParallelism, int parallelism, int keyGroupId) {
   return keyGroupId * parallelism / maxParallelism;
}

//计算key所属的keyGroupId
public static int assignToKeyGroup(Object key, int maxParallelism) {
   return computeKeyGroupForKeyHash(key.hashCode(), maxParallelism);
}

public static int computeKeyGroupForKeyHash(int keyHash, int maxParallelism) {
   return MathUtils.murmurHash(keyHash) % maxParallelism;
}

这里解释一样keyGroupId的概念,Reduce端每个task都持有一个KeyGroupRange,是一个keyGroupId的区间,这个是根据最大任务并行度来算的,例如默认最大并行度是128, keyGroupId就是从0到127,假设Reduce端有2个任务,那每个Reduce都持有一半的keyGroupId区间,task1是[0, 63], task2是[64, 127]。上游map端通过计算key所在的keyGroupId就知道这个key属于下游的那个Reduce任务了
从代码中看到最终的下游任务选择计算公式为:
(hash(key) % maxTask) * numTask / maxTasks

然后我们再来看另一种ChannelSelector:RebalancePartitioner,这种分区器是均衡分区器,代码里调用DataStream.rebalance()会使用这种分区器对数据重新进行分布。当上游出现数据倾斜时可以将数据均衡的进行分区发送到下游,它的实现如下:

//RebalancePartitioner类
public int selectChannel(SerializationDelegate<StreamRecord<T>> record) {
   nextChannelToSendTo = (nextChannelToSendTo + 1) % numberOfChannels;
   return nextChannelToSendTo;
}

可以看到这种分区器的工作就是不断轮询的选择下游的任务

4. 数据输出模型ResultPartition

public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner {

   private static final Logger LOG = LoggerFactory.getLogger(ResultPartition.class);

   private final String owningTaskName;

   private final TaskActions taskActions;

   private final JobID jobId;

   private final ResultPartitionID partitionId;

   /** Type of this partition. Defines the concrete subpartition implementation to use. */
   // 上下游之间数据的传送类型,
   // BLOCKING: 阻塞,上游数据处理完了,下游才开始处理;
   // PIPELINED: 流水线,上下游可以同时进行生产和消费;
   // PIPELINED_BOUNDED: 流水线模式,但是流量有限制,
   // 意思是在出现背压时,任务不会将大量的数据缓存到自己的buffer里,在流处理情况下都是这种类型
   private final ResultPartitionType partitionType;

   /** The subpartitions of this partition. At least one. */
   // ResultSubpartition类型数组,ResultSubpartition对应的是下游的任务,
   // 一个ResultSubpartition对应了一个下游子任务,每个ResultSubpartition
   //里会有一些buffer来缓存下游任务所要消费的数据。
   private final ResultSubpartition[] subpartitions;

   private final ResultPartitionManager partitionManager;

   // 对应批处理模式,当partition的数据可用了,这个唤醒者会触发JobManager进行下游任务部署
   private final ResultPartitionConsumableNotifier partitionConsumableNotifier;

   public final int numTargetKeyGroups;

   // 对应批处理模式,当partition的数据可用了,会触发JobManager进行下游任务部署
   private final boolean sendScheduleOrUpdateConsumersMessage;

   // 数据缓存buffer池,相当核心的成员,是一个LocalBufferPool,Map端往ResultPartition写数据时,
   // 会向LocalBufferPool申请buffer,将数据写到buffer里
   private BufferPool bufferPool;

   private boolean hasNotifiedPipelinedConsumers;

   private boolean isFinished;
  

可以看到构造方法中最重要的就是构建ResultSubpartition,如果是PIPELINED或者PIPELINED_BOUNDED,就创建PipelinedSubpartition,这种情况下数据是放在内存中的,如果是BLOCKING,就创建SpillableSubpartition,这种情况下数据会spill到磁盘上。

public ResultPartition(
   String owningTaskName,
   TaskActions taskActions, // actions on the owning task
   JobID jobId,
   ResultPartitionID partitionId,
   ResultPartitionType partitionType,
   int numberOfSubpartitions,
   int numTargetKeyGroups,
   ResultPartitionManager partitionManager,
   ResultPartitionConsumableNotifier partitionConsumableNotifier,
   IOManager ioManager,
   boolean sendScheduleOrUpdateConsumersMessage) {
   this.owningTaskName = checkNotNull(owningTaskName);
   this.taskActions = checkNotNull(taskActions);
   this.jobId = checkNotNull(jobId);
   this.partitionId = checkNotNull(partitionId);
   this.partitionType = checkNotNull(partitionType);
   this.subpartitions = new ResultSubpartition[numberOfSubpartitions];
   this.numTargetKeyGroups = numTargetKeyGroups;
   this.partitionManager = checkNotNull(partitionManager);
   this.partitionConsumableNotifier = checkNotNull(partitionConsumableNotifier);
   this.sendScheduleOrUpdateConsumersMessage = sendScheduleOrUpdateConsumersMessage;
   // Create the subpartitions.
   switch (partitionType) {
      case BLOCKING:
         for (int i = 0; i < subpartitions.length; i++) {
            subpartitions[i] = new SpillableSubpartition(i, this, ioManager);
         }
         break;
      case PIPELINED:
      case PIPELINED_BOUNDED:
         for (int i = 0; i < subpartitions.length; i++) {
            subpartitions[i] = new PipelinedSubpartition(i, this);
         }
         break;
      default:
         throw new IllegalArgumentException("Unsupported result partition type.");
   }
   // Initially, partitions should be consumed once before release.
   pin();
   LOG.debug("{}: Initialized {}", owningTaskName, this);
}

5. 子模型ResultSubpartition

下面再看ResultSubpartition,每个Map任务端的输出是ResultPartition,代表着整个Map端输出数据的一个分区,那每个Map任务又对应了多个Reduce任务,也就是ResultPartition的数据要被多个Reduce任务所消费,这样,每个ResultPartition就持有多个ResultSubpartition,每个ResultSubpartition都对应了下游的一个Reduce任务

public abstract class ResultSubpartition {
   /** The index of the subpartition at the parent partition. */
   protected final int index;
   /** The parent partition this subpartition belongs to. */
   protected final ResultPartition parent;
   /** All buffers of this subpartition. Access to the buffers is synchronized on this object. */
   protected final ArrayDeque<BufferConsumer> buffers = new ArrayDeque<>();
   /** The number of non-event buffers currently in this subpartition. */
   @GuardedBy("buffers")
   private int buffersInBacklog;
   // - Statistics ----------------------------------------------------------
   /** The total number of buffers (both data and event buffers). */
   private long totalNumberOfBuffers;

核心的成员有:
parent:所属的ResultPartition
buffers:ResultSubpartition用于缓存数据的buffer,RecordWriter将数据写到这些buffer里,然后Reduce任务会消费这些buffer数据
buffersInBacklog:ResultSubpartition中buffer的积压量,当有新buffer添加进来这个值会加1,当有buffer被消费时这个值会减1

下面看流计算和批处理默认情况下的ResultSubpartition 类型PipelinedSubpartition

class PipelinedSubpartition extends ResultSubpartition {

   private static final Logger LOG = LoggerFactory.getLogger(PipelinedSubpartition.class);

   // ------------------------------------------------------------------------

   /** The read view to consume this subpartition. */
   private PipelinedSubpartitionView readView;

   /** Flag indicating whether the subpartition has been finished. */
   private boolean isFinished;

   @GuardedBy("buffers")
   private boolean flushRequested;

   /** Flag indicating whether the subpartition has been released. */
   private volatile boolean isReleased;

核心的成员有:
readView:消费ResultSubpartition buffer的reader视图,由netty服务端向消费端发送数据时调用
flushRequested:刷新请求,上述说过,当Map端的数据生产速度较慢时,会定时刷新buffer数据,这个变量就是用来标记刷新动作的

6. 本地buffer池LocalBufferPool

class LocalBufferPool implements BufferPool {
   private static final Logger LOG = LoggerFactory.getLogger(LocalBufferPool.class);

   /** Global network buffer pool to get buffers from. */
   private final NetworkBufferPool networkBufferPool;

   /** The minimum number of required segments for this pool. */
   private final int numberOfRequiredMemorySegments;

   /**
    * The currently available memory segments. These are segments, which have been requested from
    * the network buffer pool and are currently not handed out as Buffer instances.
    *
    * 

BEWARE: Take special care with the interactions between this lock and * locks acquired before entering this class vs. locks being acquired during calls to external * code inside this class, e.g. with * {@link org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel#bufferQueue} * via the {@link #registeredListeners} callback. */ private final ArrayDeque<MemorySegment> availableMemorySegments = new ArrayDeque<MemorySegment>(); /** * Buffer availability listeners, which need to be notified when a Buffer becomes available. * Listeners can only be registered at a time/state where no Buffer instance was available. */ private final ArrayDeque<BufferListener> registeredListeners = new ArrayDeque<>(); /** Maximum number of network buffers to allocate. */ private final int maxNumberOfMemorySegments; /** The current size of this pool. */ private int currentPoolSize; /** * Number of all memory segments, which have been requested from the network buffer pool and are * somehow referenced through this pool (e.g. wrapped in Buffer instances or as available segments). */ private int numberOfRequestedMemorySegments; private boolean isDestroyed; private final Optional<BufferPoolOwner> owner;

核心成员如下:
networkBufferPool:全局的网络buffer池,LocalBufferPool中的buffer将通过networkBufferPool中获取
availableMemorySegments:LocalBufferPool中当前可用的buffer队列,已经从networkBufferPool获取,但是还没有用来缓冲数据的buffer。LocalBufferPool中的buffer形式就是MemorySegment
registeredListeners:buffer可用时的监听器,当LocalBufferPool中有可用的buffer了,就会通知这些监听器,一般在数据消费端才会使用
maxNumberOfMemorySegments:LocalBufferPool中最大的buffer数量
numberOfRequestedMemorySegments:已经从networkBufferPool中申请的buffer数量

再看创建过程
总的创建过程在Task开始启动的时候,进行初始化工作的过程中,在Task.run()方法里

//NetworkEnvironment类
public void setupPartition(ResultPartition partition) throws IOException {
   BufferPool bufferPool = null;

   try {
       //计算bufferPool的最大buffer容量,这依赖于ResultPartitionType类型,流计算都是PIPELINED_BOUNDED
      int maxNumberOfMemorySegments = partition.getPartitionType().isBounded() ?
         partition.getNumberOfSubpartitions() * networkBuffersPerChannel +
            extraNetworkBuffersPerGate : Integer.MAX_VALUE;
      // If the partition type is back pressure-free, we register with the buffer pool for
      // callbacks to release memory.
      bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfSubpartitions(),
         maxNumberOfMemorySegments,
         partition.getPartitionType().hasBackPressure() ? Optional.empty() : Optional.of(partition));
        //将这个bufferPool设置给ResultPartition 
      partition.registerBufferPool(bufferPool);

      resultPartitionManager.registerResultPartition(partition);
   } catch (Throwable t) {
     ...
   }

   taskEventDispatcher.registerPartition(partition.getPartitionId());
}

这里可以看到在流计算中,LocalBufferPool中最大的buffer数 = task对应的下游任务数 * 每个下游任务需要的buffer + 额外多分配的buffer数。每个下游任务所需buffer数默认是2,由参数taskmanager.network.memory.buffers-per-channel控制;额外多分配的buffer数默认是8,由参数taskmanager.network.memory.floating-buffers-per-gate控制。例如下游有两个reduce任务,那每个map任务的ResultPartition拥有的最大的buffer数就是2*2+8=12

我们再回到之前的RecordWriter.copyFromSerializerToTargetChannel()方法,来看一下具体的过程

7. 获取buffer

//RecordWriter类
private BufferBuilder getBufferBuilder(int targetChannel) throws IOException, InterruptedException {
   if (bufferBuilders[targetChannel].isPresent()) {
       //如果对应通道的buffer还没被写满,就直接返回了buffer
      return bufferBuilders[targetChannel].get();
   } else {
       //如果第一次获取buffer或者之前的buffer写满了,就需要新申请buffer了
      return requestNewBufferBuilder(targetChannel);
   }
}

private BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException {
   checkState(!bufferBuilders[targetChannel].isPresent() || bufferBuilders[targetChannel].get().isFinished());
   //从ResultPartition的LocalBufferPool申请新的buffer
   BufferBuilder bufferBuilder = targetPartition.getBufferProvider().requestBufferBuilderBlocking();
   bufferBuilders[targetChannel] = Optional.of(bufferBuilder);
   //申请完buffer之后将这个buffer添加到对应通道的ResultSubPartition里
   targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel);
   return bufferBuilder;
}

在申请新的buffer时会调用LocalBufferPool.requestBufferBuilderBlocking()

public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException {
   return toBufferBuilder(requestMemorySegment(true));
}

private MemorySegment requestMemorySegment(boolean isBlocking) throws InterruptedException, IOException {
   synchronized (availableMemorySegments) {
      returnExcessMemorySegments();

      boolean askToRecycle = owner.isPresent();
      //如果availableMemorySegments里有buffer,就直接从availableMemorySegments里获取了,如果没有,就等待或者从networkBufferPool申请buffer
      // fill availableMemorySegments with at least one element, wait if required
      while (availableMemorySegments.isEmpty()) {
         if (isDestroyed) {
            throw new IllegalStateException("Buffer pool is destroyed.");
         }
        //如果还没达到BufferPool的最大容量,就继续向networkBufferPool申请buffer
         if (numberOfRequestedMemorySegments < currentPoolSize) {
            final MemorySegment segment = networkBufferPool.requestMemorySegment();

            if (segment != null) {
               numberOfRequestedMemorySegments++;
               return segment;
            }
         }

         if (askToRecycle) {
            owner.get().releaseMemory(1);
         }
         //申请已经达到最大值,如果阻塞,就阻塞在这里,2秒之后再次申请buffer,或者等待被唤醒
       if (isBlocking) {
            availableMemorySegments.wait(2000);
         }
         else {
            return null;
         }
      }
    
      return availableMemorySegments.poll();
   }
}

从方法中我们也不难分析出,LocalBufferPool是一个buffer池,它的buffer从networkBufferPool中去申请,当申请达到最大数量时,不再继续申请,这也是LocalBufferPool中的buffer上限。当buffer的数据被下游消费后进行释放回收时,就把buffer放到了availableMemorySegments队列中,后面再从LocalBufferPool中申请buffer,就直接从availableMemorySegments中取了。
当LocalBufferPool中没有空闲buffer,而向networkBufferPool申请的buffer又达到上限时,申请就会被阻塞,这时写数据的过程也就被阻塞了。直到有buffer被回收

8. 将buffer添加到ResultSubpartition

当RecordWriter申请到了这个buffer之后,就可以往buffer里写数据了,而ResultPartition也会将这个buffer添加到对应的ResultSubpartition里
大致的实现如下:
1、将buffer添加到对应通道的ResultSubpartition里,ResultSubpartition将buffer添加到自己的buffer队列里,这个buffer队列就是等待被下游任务消费的数据队列
2、将ResultSubpartition的buffer积压值加1,这个积压值应用在背压过程,下游数据消费时会获取到这个积压值,来调整用于数据接收到buffer数
3、判断是否应该告知数据可用,这将使得生产端将数据发送给下游消费端,或者唤醒下游任务进行数据消费。告知的条件是ResultSubpartition里只有一个finish的buffer,也就是说当把第一个buffer添加进ResultSubpartition里时不会触发消费的,因为这时RecordWriter还没来得及向这个buffer写数据,buffer还没finish。当添加第二个buffer到ResultSubpartition里才会触发数据发送和下游任务的消费。
4、ResultSubpartition将buffer添加之后,ResultPartition会通知JobManager进行下游任务的部署或者状态变更,这种情况发生在批处理下。在流处理就没有这个步骤了

//ResultPartition
public void addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException {
   checkNotNull(bufferConsumer);

   ResultSubpartition subpartition;
   try {
      checkInProduceState();
      //选择对应通道的ResultSubpartition
      subpartition = subpartitions[subpartitionIndex];
   }
   catch (Exception ex) {
      bufferConsumer.close();
      throw ex;
   }
    
   if (subpartition.add(bufferConsumer)) {
       //唤醒pipeline模式下的消费端,在流处理时不需要。在批处理模式下这会触发下游任务部署或者状态变更
      notifyPipelinedConsumers();
   }
}

//PipelinedSubpartition
public boolean add(BufferConsumer bufferConsumer) {
   return add(bufferConsumer, false);
}

private boolean add(BufferConsumer bufferConsumer, boolean finish) {
   checkNotNull(bufferConsumer);

   final boolean notifyDataAvailable;
   synchronized (buffers) {
      if (isFinished || isReleased) {
         bufferConsumer.close();
         return false;
      }

      // Add the bufferConsumer and update the stats
      //添加到ResultSubpartition的buffers队列里
      buffers.add(bufferConsumer);
      updateStatistics(bufferConsumer);
      increaseBuffersInBacklog(bufferConsumer);
      //添加完buffer之后判断是否应该告知数据可用
      notifyDataAvailable = shouldNotifyDataAvailable() || finish;

      isFinished |= finish;
   }
    //告知数据可用,这可能会触发生产端将数据发送给下游消费任务
   if (notifyDataAvailable) {
      notifyDataAvailable();
   }

   return true;
}

private boolean shouldNotifyDataAvailable() {
    //ResultSubPartition数据可用的条件
   // Notify only when we added first finished buffer.
   return readView != null && !flushRequested && getNumberOfFinishedBuffers() == 1;
}

到此,Task数据写过程基本就分析完了,当然这里的写仅仅是写到了ResultPartition里,或者说写到了ResultSubPartition里,数据以一个个buffer的形式存在。flink task之间数据传输的过程将在以后进行具体分析

你可能感兴趣的:(flink,交互,java)