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学习课程,适合初学者阅读。由浅入深案例丰富,通俗易懂。主要涉及基础的系统操作以及工作中常用的各种服务软件的应用、部署和优化。即使是零基础的学员,只要能够坚持把所有章节都学完,也一定会受益匪浅。
相关文章
|
7月前
|
存储 监控 Serverless
Serverless 应用的监控与调试问题之Flink对于Checkpoint Barrier流动缓慢的问题要如何解决
Serverless 应用的监控与调试问题之Flink对于Checkpoint Barrier流动缓慢的问题要如何解决
|
6月前
|
运维 数据处理 数据安全/隐私保护
阿里云实时计算Flink版测评报告
该测评报告详细介绍了阿里云实时计算Flink版在用户行为分析与标签画像中的应用实践,展示了其毫秒级的数据处理能力和高效的开发流程。报告还全面评测了该服务在稳定性、性能、开发运维及安全性方面的卓越表现,并对比自建Flink集群的优势。最后,报告评估了其成本效益,强调了其灵活扩展性和高投资回报率,适合各类实时数据处理需求。
|
4月前
|
存储 分布式计算 流计算
实时计算 Flash – 兼容 Flink 的新一代向量化流计算引擎
本文介绍了阿里云开源大数据团队在实时计算领域的最新成果——向量化流计算引擎Flash。文章主要内容包括:Apache Flink 成为业界流计算标准、Flash 核心技术解读、性能测试数据以及在阿里巴巴集团的落地效果。Flash 是一款完全兼容 Apache Flink 的新一代流计算引擎,通过向量化技术和 C++ 实现,大幅提升了性能和成本效益。
1895 73
实时计算 Flash – 兼容 Flink 的新一代向量化流计算引擎
|
8月前
|
存储 监控 大数据
阿里云实时计算Flink在多行业的应用和实践
本文整理自 Flink Forward Asia 2023 中闭门会的分享。主要分享实时计算在各行业的应用实践,对回归实时计算的重点场景进行介绍以及企业如何使用实时计算技术,并且提供一些在技术架构上的参考建议。
988 7
阿里云实时计算Flink在多行业的应用和实践
|
2月前
|
消息中间件 关系型数据库 MySQL
Flink CDC 在阿里云实时计算Flink版的云上实践
本文整理自阿里云高级开发工程师阮航在Flink Forward Asia 2024的分享,重点介绍了Flink CDC与实时计算Flink的集成、CDC YAML的核心功能及应用场景。主要内容包括:Flink CDC的发展及其在流批数据处理中的作用;CDC YAML支持的同步链路、Transform和Route功能、丰富的监控指标;典型应用场景如整库同步、Binlog原始数据同步、分库分表同步等;并通过两个Demo展示了MySQL整库同步到Paimon和Binlog同步到Kafka的过程。最后,介绍了未来规划,如脏数据处理、数据限流及扩展数据源支持。
224 0
Flink CDC 在阿里云实时计算Flink版的云上实践
zdl
|
4月前
|
消息中间件 运维 大数据
大数据实时计算产品的对比测评:实时计算Flink版 VS 自建Flink集群
本文介绍了实时计算Flink版与自建Flink集群的对比,涵盖部署成本、性能表现、易用性和企业级能力等方面。实时计算Flink版作为全托管服务,显著降低了运维成本,提供了强大的集成能力和弹性扩展,特别适合中小型团队和业务波动大的场景。文中还提出了改进建议,并探讨了与其他产品的联动可能性。总结指出,实时计算Flink版在简化运维、降低成本和提升易用性方面表现出色,是大数据实时计算的优选方案。
zdl
233 56
|
3月前
|
存储 关系型数据库 BI
实时计算UniFlow:Flink+Paimon构建流批一体实时湖仓
实时计算架构中,传统湖仓架构在数据流量管控和应用场景支持上表现良好,但在实际运营中常忽略细节,导致新问题。为解决这些问题,提出了流批一体的实时计算湖仓架构——UniFlow。该架构通过统一的流批计算引擎、存储格式(如Paimon)和Flink CDC工具,简化开发流程,降低成本,并确保数据一致性和实时性。UniFlow还引入了Flink Materialized Table,实现了声明式ETL,优化了调度和执行模式,使用户能灵活调整新鲜度与成本。最终,UniFlow不仅提高了开发和运维效率,还提供了更实时的数据支持,满足业务决策需求。
|
7月前
|
SQL 消息中间件 Kafka
实时计算 Flink版产品使用问题之如何在EMR-Flink的Flink SOL中针对source表单独设置并行度
实时计算Flink版作为一种强大的流处理和批处理统一的计算框架,广泛应用于各种需要实时数据处理和分析的场景。实时计算Flink版通常结合SQL接口、DataStream API、以及与上下游数据源和存储系统的丰富连接器,提供了一套全面的解决方案,以应对各种实时计算需求。其低延迟、高吞吐、容错性强的特点,使其成为众多企业和组织实时数据处理首选的技术平台。以下是实时计算Flink版的一些典型使用合集。
|
4月前
|
SQL 运维 数据可视化
阿里云实时计算Flink版产品体验测评
阿里云实时计算Flink基于Apache Flink构建,提供一站式实时大数据分析平台,支持端到端亚秒级实时数据分析,适用于实时大屏、实时报表、实时ETL和风控监测等场景,具备高性价比、开发效率、运维管理和企业安全等优势。
|
6月前
|
人工智能 Apache 流计算
Flink Forward Asia 2024 上海站|探索实时计算新边界
Flink Forward Asia 2024 即将盛大开幕!11 月 29 至 30 日在上海举行,大会聚焦 Apache Flink 技术演进与未来规划,涵盖流式湖仓、流批一体、Data+AI 融合等前沿话题,提供近百场专业演讲。立即报名,共襄盛举!官网:https://asia.flink-forward.org/shanghai-2024/
1025 33
Flink Forward Asia 2024 上海站|探索实时计算新边界