本文分析的源码为flink 1.18.0_scala2.12版本。
目前 Flink 支持8种分区策略的实现,如下图所示:
可以在 org.apache.flink.streaming.api.datastream.DataStream 类中看到一下方法:
public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, KeySelector<T, K> keySelector)
public DataStream<T> broadcast()
public DataStream<T> shuffle()
public DataStream<T> forward()
public DataStream<T> rebalance()
public DataStream<T> rescale()
public DataStream<T> global()
可以在这些方法中看到系统所调用的分区器,分区器中有 selectChannel 方法,该方法返回一个 int 类型的数,负责选择下游算子实例的编号。
以下分别介绍 8 中分区策略:
- (1)GlobalPartitioner
数据会被分发到下游算子的第一个实例中进行处理。
选择下游算子示例编号的方式为:@Internal public class GlobalPartitioner<T> extends StreamPartitioner<T> { private static final long serialVersionUID = 1L; @Override public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { return 0; } }
- (2)ForwardPartitioner
在API层面上ForwardPartitioner应用在 DataStream上,生成一个新的 DataStream。
该Partitioner 比较特殊,用于在同一个 OperatorChain 中上下游算子之间的数据转发,实际上数据是直接传递给下游的,要求上下游并行度一样。
选择下游算子示例编号的方式为:@Internal public class ForwardPartitioner<T> extends StreamPartitioner<T> { @Override public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { return 0; } @Override public SubtaskStateMapper getDownstreamSubtaskStateMapper() { return SubtaskStateMapper.UNSUPPORTED; } @Override public SubtaskStateMapper getUpstreamSubtaskStateMapper() { return SubtaskStateMapper.UNSUPPORTED; } }
- (3)ShufflePartitioner
随机的将元素进行分区,可以确保下游的Task能够均匀地获得数据。
使用该功能的 API 如下:
选择下游算子示例编号的方式为:dataStream.shuffle();
@Internal public class ShufflePartitioner<T> extends StreamPartitioner<T> { private Random random = new Random(); @Override public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { return random.nextInt(numberOfChannels); } }
- (4)RebalancePartitioner
以 Round-robin 的方式为每个元素分配分区,确保下游的 Task 可以均匀地获得数据,避免数据倾斜。
使用该功能的 API 如下:
选择下游算子示例编号的方式为:dataStream.rebalance()
@Internal public class RebalancePartitioner<T> extends StreamPartitioner<T> { private int nextChannelToSendTo; @Override public void setup(int numberOfChannels) { super.setup(numberOfChannels); nextChannelToSendTo = ThreadLocalRandom.current().nextInt(numberOfChannels); } @Override public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { nextChannelToSendTo = (nextChannelToSendTo + 1) % numberOfChannels; return nextChannelToSendTo; } @Override public SubtaskStateMapper getDownstreamSubtaskStateMapper() { return SubtaskStateMapper.ROUND_ROBIN; } }
- (5)RescalePartitioner
根据上下游 Task 的数量进行分区, 使用 Round-robin 选择下游的一个Task 进行数据分区,如上游有2个 Source.,下游有6个 Map,那么每个 Source 会分配3个固定的下游 Map,不会向未分配给自己的分区写人数据。这一点与 ShufflePartitioner 和 RebalancePartitioner 不同, 后两者可能会写入下游任意分区。
使用该功能的 API 如下:
选择下游算子示例编号的方式为:dataStream.rescale();
@Internal public class RescalePartitioner<T> extends StreamPartitioner<T> { private int nextChannelToSendTo = -1; @Override public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { if (++nextChannelToSendTo >= numberOfChannels) { nextChannelToSendTo = 0; } return nextChannelToSendTo; } @Override public SubtaskStateMapper getDownstreamSubtaskStateMapper() { return SubtaskStateMapper.UNSUPPORTED; } @Override public SubtaskStateMapper getUpstreamSubtaskStateMapper() { return SubtaskStateMapper.UNSUPPORTED; } }
- (6)BroadcastPartitioner
将该记录广播给所有分区,即有N个分区,就把数据复制N份,每个分区1份。
使用该功能的 API 如下:
广播模式可直接在 record writer 中处理所有输出通道,因此无需通过分区器选择通道。一次其分区器中selectChannel方法是不可调用的,否则抛出异常。dataStream.broadcast();
@Internal public class BroadcastPartitioner<T> extends StreamPartitioner<T> { public int selectChannel(SerializationDelegate<StreamRecord<T>> record) { throw new UnsupportedOperationException( "Broadcast partitioner does not support select channels."); } }
- (7)KeyGroupStreamPartitioner
在API层面上,KeyGroupStreamPartitioner应用在 KeyedStream上,生成一个新的 KeyedStream。
KeyedStream根据keyGroup索引编号进行分区,会将数据按 Key 的 Hash 值输出到下游算子实例中。该分区器不是提供给用户来用的。
KeyedStream在构造Transformation的时候默认使用KeyedGroup分区形式,从而在底层上支持作业Rescale功能。 - (8)CustomPartitionerWrapper
用户自定义分区器,需要用户自己实现 Partitioner 接口,来定义自己的分区逻辑。
自定义示例如下:
通过 DataStream.partitionCustom 方法使用用户自定义的分区器。static class CustomPartitioner implements Partitioner<String> { @Override public int partition(String key, int numPartitions) { switch (key){ case "1": return 1; case "2": return 2; default: return 3; } } }