Flink -- Barrier

本文涉及的产品
实时计算 Flink 版,5000CU*H 3个月
简介:

CheckpointBarrierHandler

这个接口用于react从input channel过来的checkpoint barrier,这里可以通过不同的实现来,决定是简单的track barriers,还是要去真正的block inputs

复制代码
/**
 * The CheckpointBarrierHandler reacts to checkpoint barrier arriving from the input channels.
 * Different implementations may either simply track barriers, or block certain inputs on
 * barriers.
 */
public interface CheckpointBarrierHandler {

    /**
     * Returns the next {@link BufferOrEvent} that the operator may consume.
     * This call blocks until the next BufferOrEvent is available, ir until the stream
     * has been determined to be finished.
     * 
     * @return The next BufferOrEvent, or {@code null}, if the stream is finished.
     * @throws java.io.IOException Thrown, if the network or local disk I/O fails.
     * @throws java.lang.InterruptedException Thrown, if the thread is interrupted while blocking during
     *                                        waiting for the next BufferOrEvent to become available.
     */
    BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException;

    /**
     * Registers the given event handler to be notified on successful checkpoints.
     * 
     * @param checkpointHandler The handler to register.
     */
    void registerCheckpointEventHandler(EventListener<CheckpointBarrier> checkpointHandler);

    /**
     * Cleans up all internally held resources.
     * 
     * @throws IOException Thrown, if the cleanup of I/O resources failed.
     */
    void cleanup() throws IOException;

    /**
     * Checks if the barrier handler has buffered any data internally.
     * @return True, if no data is buffered internally, false otherwise.
     */
    boolean isEmpty();
}
复制代码

 

BarrierBuffer

 

复制代码
/**
 * The barrier buffer is {@link CheckpointBarrierHandler} that blocks inputs with barriers until
 * all inputs have received the barrier for a given checkpoint.
 * 
 * <p>To avoid back-pressuring the input streams (which may cause distributed deadlocks), the
 * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until 
 * the blocks are released.</p>
 */
public class BarrierBuffer implements CheckpointBarrierHandler {
    
    /** The gate that the buffer draws its input from */
    private final InputGate inputGate;   //输入

    /** Flags that indicate whether a channel is currently blocked/buffered */
    private final boolean[] blockedChannels;  //被blocked的channels
    
    /** The total number of channels that this buffer handles data from */
    private final int totalNumberOfInputChannels;
    
    /** To utility to write blocked data to a file channel */
    private final BufferSpiller bufferSpiller;  //为了不造成反压,对于被block的channl,不会真正的block,而是只是把数据放到buffer中

    /** The pending blocked buffer/event sequences. Must be consumed before requesting
     * further data from the input gate. */
    private final ArrayDeque<BufferSpiller.SpilledBufferOrEventSequence> queuedBuffered; //更多的没来得及处理的unblock buffer数据

    /** The sequence of buffers/events that has been unblocked and must now be consumed
     * before requesting further data from the input gate */
    private BufferSpiller.SpilledBufferOrEventSequence currentBuffered; //由bufferSpiller进行rollover产生的,已经unblock的buffer数据

    /** Handler that receives the checkpoint notifications */
    private EventListener<CheckpointBarrier> checkpointHandler; //创建checkpoint的逻辑

    /** The ID of the checkpoint for which we expect barriers */
    private long currentCheckpointId = -1L;

    /** The number of received barriers (= number of blocked/buffered channels) */
    private int numBarriersReceived;
    
    /** The number of already closed channels */
    private int numClosedChannels;
    
    /** Flag to indicate whether we have drawn all available input */
    private boolean endOfStream;
}
复制代码

 

最关键的函数,

getNextNonBlocked
复制代码
@Override
public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException {
    while (true) {
        // process buffered BufferOrEvents before grabbing new ones
        BufferOrEvent next;
        if (currentBuffered == null) { //如果currentBuffered为空,说明没有unblock的buffer数据,直接从inputGate读取
            next = inputGate.getNextBufferOrEvent();
        }
        else {
            next = currentBuffered.getNext(); //从currentBuffered读
            if (next == null) {  //如果读到的为空,说明currentBuffered已经为空
                completeBufferedSequence(); //清空当前的currentBuffered,看看queuedBuffered中还有没有需要处理的buffer
                return getNextNonBlocked();
            }
        }
        
        if (next != null) {
            if (isBlocked(next.getChannelIndex())) { //如果这个channel仍然是被block的
                // if the channel is blocked we, we just store the BufferOrEvent
                bufferSpiller.add(next); //那么我们只是把这个BufferOrEvent放到bufferSpiller里面
            }
            else if (next.isBuffer()) { //如果没有被block,就处理该条数据,如果是buffer即真实数据,那么直接返回该数据
                return next;
            }
            else if (next.getEvent().getClass() == CheckpointBarrier.class) { //如果是CheckpointBarrier
                if (!endOfStream) {
                    // process barriers only if there is a chance of the checkpoint completing
                    processBarrier((CheckpointBarrier) next.getEvent(), next.getChannelIndex()); //那么调用processBarrier,后面具体分析
                }
            }
            else {
                if (next.getEvent().getClass() == EndOfPartitionEvent.class) {
                    numClosedChannels++;
                    // no chance to complete this checkpoint
                    releaseBlocks(); //因为某个channel close了,那就永远也无法从这个channel获取barrier了,所以releaseBlocks
                }
                return next;
            }
        }
        else if (!endOfStream) {
            // end of stream. we feed the data that is still buffered
            endOfStream = true;
            releaseBlocks();//流结束了,所以也需要releaseBlocks
            return getNextNonBlocked();
        }
        else {
            return null;
        }
    }
}
复制代码

 

其中两个函数比较重要processBarrier和releaseBlocks

processBarrier

复制代码
private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws IOException {
    final long barrierId = receivedBarrier.getId(); //取出全局barrier id

    if (numBarriersReceived > 0) { //如果之前收到过barrier
        // subsequent barrier of a checkpoint.
        if (barrierId == currentCheckpointId) {  //看下刚收到的和之前的barrierid是否一样
            // regular case
            onBarrier(channelIndex); //如果一样调用onBarrier
        }
        else if (barrierId > currentCheckpointId) { //如果大于currentCheckpointId,说明这个id已经过期了,因为在一个channel上,barrier id应该是按序发送的
            // we did not complete the current checkpoint
            LOG.warn("Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " +
                    "Skipping current checkpoint.", barrierId, currentCheckpointId);

            releaseBlocks(); //既然这个barrier已经过期,所以releaseBlocks()
            currentCheckpointId = barrierId; //设置新的barrierId
            onBarrier(channelIndex);
        }
        else { //忽略已过期的barrier
            // ignore trailing barrier from aborted checkpoint
            return;
        }
        
    }
    else if (barrierId > currentCheckpointId) { //新的barrier
        // first barrier of a new checkpoint
        currentCheckpointId = barrierId;
        onBarrier(channelIndex);
    }
    else {
        // trailing barrier from previous (skipped) checkpoint
        return;
    }

    // check if we have all barriers
    if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) { //如果我们已经集齐所有的barrier
        if (LOG.isDebugEnabled()) {
            LOG.debug("Received all barrier, triggering checkpoint {} at {}",
                    receivedBarrier.getId(), receivedBarrier.getTimestamp());
        }

        if (checkpointHandler != null) {
            checkpointHandler.onEvent(receivedBarrier); //触发生成checkpoint
        }
        
        releaseBlocks(); 调用releaseBlocks
    }
}
复制代码

 

这里的onEvent,在StreamTask中定义,

复制代码
protected final EventListener<CheckpointBarrier> getCheckpointBarrierListener() {
    return new EventListener<CheckpointBarrier>() {
        @Override
        public void onEvent(CheckpointBarrier barrier) {
            try {
                triggerCheckpoint(barrier.getId(), barrier.getTimestamp()); //做checkpoint
            }
            catch (Exception e) {
                throw new RuntimeException("Error triggering a checkpoint as the result of receiving checkpoint barrier", e);
            }
        }
    };
}
复制代码

 

onBarrier,只是置标志位和计数,比较简单

复制代码
private void onBarrier(int channelIndex) throws IOException {
    if (!blockedChannels[channelIndex]) {
        blockedChannels[channelIndex] = true;
        numBarriersReceived++;
        
        if (LOG.isDebugEnabled()) {
            LOG.debug("Received barrier from channel " + channelIndex);
        }
    }
    else {
        throw new IOException("Stream corrupt: Repeated barrier for same checkpoint and input stream");
    }
}
复制代码

 

releaseBlocks

复制代码
/**
 * Releases the blocks on all channels. Makes sure the just written data
 * is the next to be consumed.
 */
private void releaseBlocks() throws IOException {

    for (int i = 0; i < blockedChannels.length; i++) {
        blockedChannels[i] = false;
    }
    numBarriersReceived = 0;

    if (currentBuffered == null) { //理论上,在调用releaseBlocks前,所有channel都是处于blocked状态,所以currentBuffered应该为空
        // common case: no more buffered data
        currentBuffered = bufferSpiller.rollOver(); //把block期间buffer的数据文件,设为currentBuffered
        if (currentBuffered != null) {
            currentBuffered.open();
        }
    }
    else { //不为空,是uncommon的case
        // uncommon case: buffered data pending
        // push back the pending data, if we have any
        
        // since we did not fully drain the previous sequence, we need to allocate a new buffer for this one
        BufferSpiller.SpilledBufferOrEventSequence bufferedNow = bufferSpiller.rollOverWithNewBuffer();
        if (bufferedNow != null) {
            bufferedNow.open();
            queuedBuffered.addFirst(currentBuffered); //currentBuffered不为空,所以先把当前的放到queuedBuffered里面
            currentBuffered = bufferedNow;
        }
    }
}
复制代码

 

看下BufferSpiller

复制代码
/**
 * Creates a new buffer spiller, spilling to one of the I/O manager's temp directories.
 * 
 * @param ioManager The I/O manager for access to teh temp directories.
 * @param pageSize The page size used to re-create spilled buffers.
 * @throws IOException Thrown if the temp files for spilling cannot be initialized.
 */
public BufferSpiller(IOManager ioManager, int pageSize) throws IOException {
    this.pageSize = pageSize;
    
    this.readBuffer = ByteBuffer.allocateDirect(READ_BUFFER_SIZE);
    this.readBuffer.order(ByteOrder.LITTLE_ENDIAN);
    
    this.headBuffer = ByteBuffer.allocateDirect(16);
    this.headBuffer.order(ByteOrder.LITTLE_ENDIAN);
    
    this.sources = new ByteBuffer[] { this.headBuffer, null }; //sources是由headBuffer和contents组成的
    
    File[] tempDirs = ioManager.getSpillingDirectories();
    this.tempDir = tempDirs[DIRECTORY_INDEX.getAndIncrement() % tempDirs.length];
    
    byte[] rndBytes = new byte[32];
    new Random().nextBytes(rndBytes);
    this.spillFilePrefix = StringUtils.byteToHexString(rndBytes) + '.';
    
    // prepare for first contents
    createSpillingChannel();
}

private void createSpillingChannel() throws IOException { //打开用于写buffer的文件
    currentSpillFile = new File(tempDir, spillFilePrefix + (fileCounter++) +".buffer");
    currentChannel = new RandomAccessFile(currentSpillFile, "rw").getChannel();
}
复制代码

 

主要的function,

add,加BufferOrEvent

复制代码
/**
 * Adds a buffer or event to the sequence of spilled buffers and events.
 * 
 * @param boe The buffer or event to add and spill.
 * @throws IOException Thrown, if the buffer of event could not be spilled.
 */
public void add(BufferOrEvent boe) throws IOException {
    hasWritten = true;
    try {
        ByteBuffer contents;
        if (boe.isBuffer()) { //分为buffer或event来提取contents
            Buffer buf = boe.getBuffer();
            contents = buf.getMemorySegment().wrap(0, buf.getSize());
        }
        else {
            contents = EventSerializer.toSerializedEvent(boe.getEvent());
        }
        
        headBuffer.clear(); //更新headBuffer
        headBuffer.putInt(boe.getChannelIndex());
        headBuffer.putInt(contents.remaining());
        headBuffer.put((byte) (boe.isBuffer() ? 0 : 1));
        headBuffer.flip();
        
        sources[1] = contents; //为什么加在1,因为0是headBuffer
        currentChannel.write(sources); //写入文件
    }
    finally {
        if (boe.isBuffer()) {
            boe.getBuffer().recycle();
        }
    }
}
复制代码

 

rollOverInternal,把当前的spill文件返回, 生成新的spill文件

复制代码
private SpilledBufferOrEventSequence rollOverInternal(boolean newBuffer) throws IOException {
    if (!hasWritten) {
        return null;
    }
    
    ByteBuffer buf;
    if (newBuffer) { //newBuffer的区别是,是否重新创建ByteBuffer还是直接用readBuffer
        buf = ByteBuffer.allocateDirect(READ_BUFFER_SIZE);
        buf.order(ByteOrder.LITTLE_ENDIAN);
    } else {
        buf = readBuffer;
    }
    
    // create a reader for the spilled data
    currentChannel.position(0L);
    SpilledBufferOrEventSequence seq = 
            new SpilledBufferOrEventSequence(currentSpillFile, currentChannel, buf, pageSize); //把当前的spill文件封装成SpilledBufferOrEventSequence
    
    // create ourselves a new spill file
    createSpillingChannel(); //生成新的spill文件
    
    hasWritten = false;
    return seq;
}
复制代码

 

对于SpilledBufferOrEventSequence,主要是提供读取的api,所以关键的函数是getNext

复制代码
/**
 * This class represents a sequence of spilled buffers and events, created by the
 * {@link BufferSpiller}. The sequence of buffers and events can be read back using the
 * method {@link #getNext()}.
 */

public static class SpilledBufferOrEventSequence {


    /**
     * Gets the next BufferOrEvent from the spilled sequence, or {@code null}, if the
     * sequence is exhausted.
     *         
     * @return The next BufferOrEvent from the spilled sequence, or {@code null} (end of sequence).
     * @throws IOException Thrown, if the reads failed, of if the byte stream is corrupt.
     */
    public BufferOrEvent getNext() throws IOException {
        if (buffer.remaining() < HEADER_LENGTH) {
            buffer.compact();
            
            while (buffer.position() < HEADER_LENGTH) {
                if (fileChannel.read(buffer) == -1) { //从文件channel你们把数据读到buffer中
                    if (buffer.position() == 0) {
                        // no trailing data
                        return null;
                    } else {
                        throw new IOException("Found trailing incomplete buffer or event");
                    }
                }
            }
            
            buffer.flip();
        }
        
        final int channel = buffer.getInt();
        final int length = buffer.getInt();
        final boolean isBuffer = buffer.get() == 0;
        
        
        if (isBuffer) { //如果是buffer
            // deserialize buffer

            MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(pageSize); //创建 MemorySegment,这里是allocate unpooled的segment
            
            int segPos = 0;
            int bytesRemaining = length;
            
            while (true) {
                int toCopy = Math.min(buffer.remaining(), bytesRemaining);
                if (toCopy > 0) {
                    seg.put(segPos, buffer, toCopy); //将buffer中的数据写入MemorySegment
                    segPos += toCopy;
                    bytesRemaining -= toCopy;
                }
                
                if (bytesRemaining == 0) {
                    break;
                }
                else {
                    buffer.clear();
                    if (fileChannel.read(buffer) == -1) {
                        throw new IOException("Found trailing incomplete buffer");
                    }
                    buffer.flip();
                }
            }
            
            
            Buffer buf = new Buffer(seg, FreeingBufferRecycler.INSTANCE); //将MemorySegment封装成Buffer
            buf.setSize(length);
            
            return new BufferOrEvent(buf, channel);
        }
        else { //如果是event
            // deserialize event
            if (buffer.remaining() < length) {
                buffer.compact();

                while (buffer.position() < length) {
                    if (fileChannel.read(buffer) == -1) {
                        throw new IOException("Found trailing incomplete event");
                    }
                }

                buffer.flip();
            }

            int oldLimit = buffer.limit();
            buffer.limit(buffer.position() + length);
            AbstractEvent evt = EventSerializer.fromSerializedEvent(buffer, getClass().getClassLoader()); //将buffer封装成event
            buffer.limit(oldLimit);
            
            return new BufferOrEvent(evt, channel);
        }
    }
}
复制代码

 

BarrierTracker,这个比Barrier buffer的实现简单的多,

因为不会去block input channel,所以无法实现exactly once,只能实现at-least once

复制代码
/**
 * The BarrierTracker keeps track of what checkpoint barriers have been received from
 * which input channels. Once it has observed all checkpoint barriers for a checkpoint ID,
 * it notifies its listener of a completed checkpoint.
 * 
 * <p>Unlike the {@link BarrierBuffer}, the BarrierTracker does not block the input
 * channels that have sent barriers, so it cannot be used to gain "exactly-once" processing
 * guarantees. It can, however, be used to gain "at least once" processing guarantees.</p>
 * 
 * <p>NOTE: This implementation strictly assumes that newer checkpoints have higher checkpoint IDs.</p>
 */
public class BarrierTracker implements CheckpointBarrierHandler {

    @Override
    public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException {
        while (true) {
            BufferOrEvent next = inputGate.getNextBufferOrEvent();
            if (next == null) {
                return null;
            }
            else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) { //如果是数据就直接返回
                return next;
            }
            else {
                processBarrier((CheckpointBarrier) next.getEvent()); //如果是barrier就处理
            }
        }
    }

    private void processBarrier(CheckpointBarrier receivedBarrier) {
        
        // general path for multiple input channels
        final long barrierId = receivedBarrier.getId();

        // find the checkpoint barrier in the queue of bending barriers
        CheckpointBarrierCount cbc = null;
        int pos = 0;
        
        for (CheckpointBarrierCount next : pendingCheckpoints) { //找找看,这个barrier是否直接收到过
            if (next.checkpointId == barrierId) {
                cbc = next;
                break;
            }
            pos++;
        }
        
        if (cbc != null) { //如果收到过
            // add one to the count to that barrier and check for completion
            int numBarriersNew = cbc.incrementBarrierCount(); //计数加一
            if (numBarriersNew == totalNumberOfInputChannels) { //判断是否所有的barrier已经到全了
                // checkpoint can be triggered
                // first, remove this checkpoint and all all prior pending
                // checkpoints (which are now subsumed)
                for (int i = 0; i <= pos; i++) {
                    pendingCheckpoints.pollFirst(); //当一个checkpoint被触发时,prior的所有checkpoint就已经过期了,也一起remove掉
                }
                
                // notify the listener
                if (checkpointHandler != null) {
                    checkpointHandler.onEvent(receivedBarrier); //如果有checkpoint handler,就调用进行check point
                }
            }
        }
        else { //新的barrier
            // first barrier for that checkpoint ID
            // add it only if it is newer than the latest checkpoint.
            // if it is not newer than the latest checkpoint ID, then there cannot be a
            // successful checkpoint for that ID anyways
            if (barrierId > latestPendingCheckpointID) {
                latestPendingCheckpointID = barrierId;
                pendingCheckpoints.addLast(new CheckpointBarrierCount(barrierId));
                
                // make sure we do not track too many checkpoints
                if (pendingCheckpoints.size() > MAX_CHECKPOINTS_TO_TRACK) {
                    pendingCheckpoints.pollFirst(); //删除过多的checkpoints
                }
            }
        }
    }
}
复制代码
相关实践学习
基于Hologres轻松玩转一站式实时仓库
本场景介绍如何利用阿里云MaxCompute、实时计算Flink和交互式分析服务Hologres开发离线、实时数据融合分析的数据大屏应用。
Linux入门到精通
本套课程是从入门开始的Linux学习课程,适合初学者阅读。由浅入深案例丰富,通俗易懂。主要涉及基础的系统操作以及工作中常用的各种服务软件的应用、部署和优化。即使是零基础的学员,只要能够坚持把所有章节都学完,也一定会受益匪浅。
相关文章
|
3月前
|
消息中间件 Kafka Apache
Apache Flink 是一个开源的分布式流处理框架
Apache Flink 是一个开源的分布式流处理框架
487 5
|
5月前
|
SQL 运维 API
Apache Flink 学习教程----持续更新
Apache Flink 学习教程----持续更新
239 0
|
2月前
|
SQL Java API
官宣|Apache Flink 1.19 发布公告
Apache Flink PMC(项目管理委员)很高兴地宣布发布 Apache Flink 1.19.0。
1376 2
官宣|Apache Flink 1.19 发布公告
|
2月前
|
SQL Apache 流计算
Apache Flink官方网站提供了关于如何使用Docker进行Flink CDC测试的文档
【2月更文挑战第25天】Apache Flink官方网站提供了关于如何使用Docker进行Flink CDC测试的文档
150 3
|
2月前
|
XML Java Apache
Apache Flink自定义 logback xml配置
Apache Flink自定义 logback xml配置
152 0
|
2月前
|
消息中间件 Java Kafka
Apache Hudi + Flink作业运行指南
Apache Hudi + Flink作业运行指南
86 1
|
2月前
|
缓存 分布式计算 Apache
Apache Hudi与Apache Flink更好地集成,最新方案了解下?
Apache Hudi与Apache Flink更好地集成,最新方案了解下?
62 0
|
2月前
|
监控 Apache 开发工具
Apache Flink 1.12.2集成Hudi 0.9.0运行指南
Apache Flink 1.12.2集成Hudi 0.9.0运行指南
67 0
|
2月前
|
SQL Java Apache
超详细步骤!整合Apache Hudi + Flink + CDH
超详细步骤!整合Apache Hudi + Flink + CDH
92 0
|
2月前
|
SQL 消息中间件 Kafka
使用 Apache Flink 和 Apache Hudi 创建低延迟数据湖管道
使用 Apache Flink 和 Apache Hudi 创建低延迟数据湖管道
43 3
使用 Apache Flink 和 Apache Hudi 创建低延迟数据湖管道