Alink漫谈(五) : 迭代计算和Superstep--2
- 0x00 摘要
- 0x05 深入Flink源码和runtime来验证
- 5.1 向Flink提交Job
- 5.2 生成JobGraph
- 5.3 迭代对应的Task
- 5.4 superstep
- 0x06 结合KMeans代码看superset
- 6.1 K-means算法概要
- 6.2 KMeansPreallocateCentroid
- 6.3 KMeansAssignCluster 和 KMeansUpdateCentroids
- 6.4 KMeansOutputModel
- 0xFF 参考
0x00 摘要
Alink 是阿里巴巴基于实时计算引擎 Flink 研发的新一代机器学习算法平台,是业界首个同时支持批式算法、流式算法的机器学习平台。迭代算法在很多数据分析领域会用到,比如机器学习或者图计算。本文将通过Superstep入手看看Alink是如何利用Flink迭代API来实现具体算法。
因为Alink的公开资料太少,所以以下均为自行揣测,肯定会有疏漏错误,希望大家指出,我会随时更新。
0x05 深入Flink源码和runtime来验证
我们需要深入到Flink内部去挖掘验证,如果大家有兴趣,可以参见下面调用栈,自己添加断点来研究。
execute:56, LocalExecutor (org.apache.flink.client.deployment.executors)
executeAsync:944, ExecutionEnvironment (org.apache.flink.api.java)
execute:860, ExecutionEnvironment (org.apache.flink.api.java)
execute:844, ExecutionEnvironment (org.apache.flink.api.java)
collect:413, DataSet (org.apache.flink.api.java)
sinkFrom:44, PrintBatchOp (com.alibaba.alink.operator.batch.utils)
sinkFrom:20, PrintBatchOp (com.alibaba.alink.operator.batch.utils)
linkFrom:31, BaseSinkBatchOp (com.alibaba.alink.operator.batch.sink)
linkFrom:17, BaseSinkBatchOp (com.alibaba.alink.operator.batch.sink)
link:89, BatchOperator (com.alibaba.alink.operator.batch)
linkTo:239, BatchOperator (com.alibaba.alink.operator.batch)
print:337, BatchOperator (com.alibaba.alink.operator.batch)
main:35, KMeansExample (com.alibaba.alink)
5.1 向Flink提交Job
Alink和Flink构建联系,是在print调用中完成的。因为是本地调试,Flink会启动一个miniCluster,然后会做如下操作。
-
首先生成执行计划Plan。Plan以数据流形式来表示批处理程序,但它只是批处理程序最初的表示,然后计划会被优化以生成更高效的方案OptimizedPlan。
-
然后,计划被编译生成JobGraph。这个图是要交给flink去生成task的图。
-
生成一系列配置。
-
将JobGraph和配置交给flink集群去运行。如果不是本地运行的话,还会把jar文件通过网络发给其他节点。
-
以本地模式运行的话,可以看到启动过程,如启动性能度量、web模块、JobManager、ResourceManager、taskManager等等。
当我们看到了submitJob调用,就知道KMeans代码已经和Flink构建了联系。
@Internal
public class LocalExecutor implements PipelineExecutor {
public static final String NAME = "local";
@Override
public CompletableFuture<JobClient> execute(Pipeline pipeline, Configuration configuration) throws Exception {
// we only support attached execution with the local executor.
checkState(configuration.getBoolean(DeploymentOptions.ATTACHED));
final JobGraph jobGraph = getJobGraph(pipeline, configuration);
final MiniCluster miniCluster = startMiniCluster(jobGraph, configuration);
final MiniClusterClient clusterClient = new MiniClusterClient(configuration, miniCluster);
CompletableFuture<JobID> jobIdFuture = clusterClient.submitJob(jobGraph);
jobIdFuture
.thenCompose(clusterClient::requestJobResult)
.thenAccept((jobResult) -> clusterClient.shutDownCluster());
return jobIdFuture.thenApply(jobID ->
new ClusterClientJobClientAdapter<>(() -> clusterClient, jobID));
}
5.2 生成JobGraph
生成jobGraph的具体流程是:
-
IterativeDataSet.closeWith会生成一个BulkIterationResultSet。
-
PrintBatchOp.sinkFrom中会调用到ExecutionEnvironment.executeAsync
-
调用createProgramPlan构建一个Plan
-
OperatorTranslation.translate函数发现
if (dataSet instanceof BulkIterationResultSet),则调用translateBulkIteration(bulkIterationResultSet); -
这时候生成了执行计划Plan
-
ExecutionEnvironment.executeAsync调用LocalExecutor.execute
-
然后调用FlinkPipelineTranslationUtil.getJobGraph来生成jobGraph
-
GraphCreatingVisitor.preVisit中会判断
if (c instanceof BulkIterationBase),以生成BulkIterationNode -
PlanTranslator.translateToJobGraph会调用到JobGraphGenerator.compileJobGraph,最终调用到createBulkIterationHead就生成了迭代处理的Head。
-
最后将jobGraph提交给Cluster ,jobGraph 变形为 ExceutionGraph在JM和TM上执行。
5.3 迭代对应的Task
前面代码中,getJobGraph函数作用是生成了job graph。
然后 JobManager 根据 JobGraph 生成 ExecutionGraph。ExecutionGraph 是 JobGraph 的并行化版本,是调度层最核心的数据结构。
最后 JobManager 根据 ExecutionGraph 对 Job 进行调度后,在各个TaskManager 上部署 Task。
所以我们需要看看最终运行时候,迭代API对应着哪些Task。
针对IterativeDataSet,即superstep-based Bulk Iterate,Flink生成了如下的task。
-
IterationHeadTask
-
IterationIntermediateTask
-
IterationTailTask
-
IterationSynchronizationSinkTask
5.3.1 IterationHeadTask
IterationHeadTask主要作用是协调一次迭代。
它会读取初始输入,和迭代Tail建立一个BlockingBackChannel。在成功处理输入之后,它会发送EndOfSuperstep事件给自己的输出。它在每次superstep之后会联系 synchronization task,等到自己收到一个用来同步的AllWorkersDoneEvent。AllWorkersDoneEvent表示所有其他的heads已经完成了自己的迭代。
下一次迭代时候,上一次迭代中tail的输出就经由backchannel传输,形成了head的输入。何时进入到下一个迭代,是由HeadTask完成的。一旦迭代完成,head将发送TerminationEvent给所有和它关联的task,告诉他们shutdown。
barrier.waitForOtherWorkers();
if (barrier.terminationSignaled()) {
requestTermination();
nextStepKickoff.signalTermination();
} else {
incrementIterationCounter();
String[] globalAggregateNames = barrier.getAggregatorNames();
Value[] globalAggregates = barrier.getAggregates();
aggregatorRegistry.updateGlobalAggregatesAndReset(globalAggregateNames, globalAggregates);
// 在这里发起下一次Superstep。
nextStepKickoff.triggerNextSuperstep();
}
}
IterationHeadTask是在JobGraphGenerator.createBulkIterationHead中构建的。其例子如下:
"PartialSolution (Bulk Iteration) (org.apache.flink.runtime.iterative.task.IterationHeadTask)"
5.3.2 IterationIntermediateTask
IterationIntermediateTask是superstep中间段的task,其将传输EndOfSuperstepEvent和TerminationEvent给所有和它关联的tasks。此外,IterationIntermediateTask能更新the workset或者the solution set的迭代状态。
如果迭代状态被更新,本task的输出将传送回IterationHeadTask,在这种情况下,本task将作为head再次被安排。
IterationIntermediateTask的例子如下:
"MapPartition (computation@KMeansUpdateCentroids) (org.apache.flink.runtime.iterative.task.IterationIntermediateTask)"
"Combine (SUM(0), at kMeansPlusPlusInit(KMeansInitCentroids.java:135) (org.apache.flink.runtime.iterative.task.IterationIntermediateTask)"
"MapPartition (AllReduceSend) (org.apache.flink.runtime.iterative.task.IterationIntermediateTask)"
"Filter (Filter at kMeansPlusPlusInit(KMeansInitCentroids.java:130)) (org.apache.flink.runtime.iterative.task.IterationIntermediateTask)"
5.3.3 IterationTailTask
IterationTailTask是迭代的最末尾。如果迭代状态被更新,本task的输出将通过BlockingBackChannel传送回IterationHeadTask,反馈给迭代头就意味着一个迭代完整逻辑的完成,那么就可以关闭这个迭代闭合环了。这种情况下,本task将在head所在的实例上重新被调度。
这里有几个关键点需要注意:
如何和Head建立联系
Flink有一个BlockingQueueBroker类,这是一个阻塞式的队列代理,它的作用是对迭代并发进行控制。Broker是单例的,迭代头任务和尾任务会生成同样的broker ID,所以头尾在同一个JVM中会基于相同的dataChannel进行通信。dataChannel由迭代头创建。
IterationHeadTask中会生成BlockingBackChannel,这是一个容量为1的阻塞队列。
// 生成channel
BlockingBackChannel backChannel = new BlockingBackChannel(new SerializedUpdateBuffer(segments, segmentSize, this.getIOManager()));
// 然后block在这里,等待Tail
superstepResult = backChannel.getReadEndAfterSuperstepEnded();
IterationTailTask则是如下:
// 在基类得到channel,因为是单例,所以会得到同一个
worksetBackChannel = BlockingBackChannelBroker.instance().getAndRemove(brokerKey());
// notify iteration head if responsible for workset update 在这里通知Head
worksetBackChannel.notifyOfEndOfSuperstep();
而两者都是利用如下办法来建立联系,在同一个subtask中会使用同一个brokerKey,这样首尾就联系起来了。
public String brokerKey() {
if (this.brokerKey == null) {
int iterationId = this.config.getIterationId();
this.brokerKey = this.getEnvironment().getJobID().toString() + '#' + iterationId + '#' + this.getEnvironment().getTaskInfo().getIndexOfThisSubtask();
}
return this.brokerKey;
}
如何把用户返回的数值传给Head
这是通过output.collect来完成的。
首先,在Tail初始化时候,会生成一个outputCollector,这个outputCollector会被设置为本task的输出outputCollector。这样就保证了用户函数的输出都会转流到outputCollector。
而outputCollector的输出就是worksetBackChannel的输出,这里设置为同一个instance。这样用户输出就输出到backChannel中。
@Override
protected void initialize() throws Exception {
super.initialize();
// set the last output collector of this task to reflect the iteration tail state update:
// a) workset update,
// b) solution set update, or
// c) merged workset and solution set update
Collector<OT> outputCollector = null;
if (isWorksetUpdate) {
// 生成一个outputCollector
outputCollector = createWorksetUpdateOutputCollector();
// we need the WorksetUpdateOutputCollector separately to count the collected elements
if (isWorksetIteration) {
worksetUpdateOutputCollector = (WorksetUpdateOutputCollector<OT>) outputCollector;
}
}
......
// 把outputCollector设置为本task的输出
setLastOutputCollector(outputCollector);
}
outputCollector的输出就是worksetBackChannel的输出buffer,这里设置为同一个instance。
protected Collector<OT> createWorksetUpdateOutputCollector(Collector<OT> delegate) {
DataOutputView outputView = worksetBackChannel.getWriteEnd();
TypeSerializer<OT> serializer = getOutputSerializer();
return new WorksetUpdateOutputCollector<OT>(outputView, serializer, delegate);
}
运行时候如下:
@Override
public void run() throws Exception {
SuperstepKickoffLatch nextSuperStepLatch = SuperstepKickoffLatchBroker.instance().get(brokerKey());
while (this.running && !terminationRequested()) {
// 用户在这里输出,最后会输出到output.collect,也就是worksetBackChannel的输出buffer。
super.run();
// 这时候以及输出到channel完毕,只是通知head进行读取。
if (isWorksetUpdate) {
// notify iteration head if responsible for workset update
worksetBackChannel.notifyOfEndOfSuperstep();
} else if (isSolutionSetUpdate) {
// notify iteration head if responsible for solution set update
solutionSetUpdateBarrier.notifySolutionSetUpdate();
}
...
}
IterationTailTask例子如下:
"Pipe (org.apache.flink.runtime.iterative.task.IterationTailTask)"
5.3.4 IterationSynchronizationSinkTask
IterationSynchronizationSinkTask作用是同步所有的iteration heads,IterationSynchronizationSinkTask被是实现成一个 output task。其只是用来协调,不处理任何数据。
在每一次superstep,IterationSynchronizationSinkTask只是等待直到它从每一个head都收到一个WorkerDoneEvent。这表示下一次superstep可以开始了。
这里需要注意的是 SynchronizationSinkTask 如何等待各个并行度的headTask。比如Flink的并行度是5,那么SynchronizationSinkTask怎么做到等待这5个headTask。
在IterationSynchronizationSinkTask中,注册了SyncEventHandler来等待head的WorkerDoneEvent。
this.eventHandler = new SyncEventHandler(numEventsTillEndOfSuperstep, this.aggregators, this.getEnvironment().getUserClassLoader());
this.headEventReader.registerTaskEventListener(this.eventHandler, WorkerDoneEvent.class);
在SyncEventHandler中,我们可以看到,在构建时候,numberOfEventsUntilEndOfSuperstep就被设置为并行度,每次收到一个WorkerDoneEvent,workerDoneEventCounter就递增,当等于numberOfEventsUntilEndOfSuperstep,即并行度时候,就说明本次superstep中,所有headtask都成功了。
private void onWorkerDoneEvent(WorkerDoneEvent workerDoneEvent) {
if (this.endOfSuperstep) {
throw new RuntimeException("Encountered WorderDoneEvent when still in End-of-Superstep status.");
} else {
// 每次递增
++this.workerDoneEventCounter;
String[] aggNames = workerDoneEvent.getAggregatorNames();
Value[] aggregates = workerDoneEvent.getAggregates(this.userCodeClassLoader);
if (aggNames.length != aggregates.length) {
throw new RuntimeException("Inconsistent WorkerDoneEvent received!");
} else {
for(int i = 0; i < aggNames.length; ++i) {
Aggregator<Value> aggregator = (Aggregator)this.aggregators.get(aggNames[i]);
aggregator.aggregate(aggregates[i]);
}
// numberOfEventsUntilEndOfSuperstep就是并行度,等于并行度时候就说明所有head都成功了。
if (this.workerDoneEventCounter % this.numberOfEventsUntilEndOfSuperstep == 0) {
this.endOfSuperstep = true;
Thread.currentThread().interrupt();
}
}
}
}
IterationSynchronizationSinkTask的例子如下:
"Sync (BulkIteration (Bulk Iteration)) (org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask)"
5.4 superstep
综上所述,我们最终得到superstep如下:
***** 文字描述如下 *****
每次迭代都是一个superstep
每次迭代中有若干subtask在不同的partition上分别执行step
每个step有一个HeadTask,若干IntermediateTask,一个TailTask
每个superstep有一个SynchronizationSinkTask
***** 伪代码大致如下 *****
for maxIter :
begin superstep
for maxSubTask :
begin step
IterationHeadTask
IterationIntermediateTask
IterationIntermediateTask
...
IterationIntermediateTask
IterationIntermediateTask
IterationTailTask
end step
IterationSynchronizationSinkTask
end superstep
0x06 结合KMeans代码看superset
6.1 K-means算法概要
K-means算法的过程,为了尽量不用数学符号,所以描述的不是很严谨,大概就是这个意思,“物以类聚、人以群分”:
-
首先输入k的值,即我们希望将数据集经过聚类得到k个分组。
-
从数据集中随机选择k个数据点作为初始大哥(质心,Centroid)
-
对集合中每一个小弟,计算与每一个大哥的距离(距离的含义后面会讲),离哪个大哥距离近,就跟定哪个大哥。
-
这时每一个大哥手下都聚集了一票小弟,这时候召开人民代表大会,每一群选出新的大哥(其实是通过算法选出新的质心)。
-
如果新大哥和老大哥之间的距离小于某一个设置的阈值(表示重新计算的质心的位置变化不大,趋于稳定,或者说收敛),可以认为我们进行的聚类已经达到期望的结果,算法终止。
-
如果新大哥和老大哥距离变化很大,需要迭代3~5步骤。
6.2 KMeansPreallocateCentroid
KMeansPreallocateCentroid也是superstep一员,但是只有context.getStepNo() == 1的时候,才会进入实际业务逻辑,预分配Centroid。当superstep为大于1的时候,本task会执行,但不会进入具体业务代码。
public class KMeansPreallocateCentroid extends ComputeFunction {
private static final Logger LOG = LoggerFactory.getLogger(KMeansPreallocateCentroid.class);
@Override
public void calc(ComContext context) {
// 每次superstep都会进到这里
LOG.info(" KMeansPreallocateCentroid 我每次都会进的呀 ");
if (context.getStepNo() == 1) {
// 实际预分配业务只进入一次
}
}
}
6.3 KMeansAssignCluster 和 KMeansUpdateCentroids
KMeansAssignCluster 作用是为每个点(point)计算最近的聚类中心,为每个聚类中心的点坐标的计数和求和。
KMeansUpdateCentroids 作用是基于计算出来的点计数和坐标,计算新的聚类中心。
Alink在整个计算过程中维护一个特殊节点来记住待求中心点当前的结果。
这就是为啥迭代时候需要区分奇数次和偶数次的原因了。奇数次就表示老大哥,偶数次就表示新大哥。每次superstep只会计算一批大哥,留下另外一批大哥做距离比对。
另外要注意的一点是:普通的迭代计算,是通过Tail给Head回传用户数据,但是KMeans这里的实现并没有采用这个办法,而是把计算出来的中心点都存在共享变量中,在各个intermediate之间互相交互。
public class KMeansAssignCluster extends ComputeFunction {
public void calc(ComContext context) {
......
if (context.getStepNo() % 2 == 0) {
stepNumCentroids = context.getObj(KMeansTrainBatchOp.CENTROID1);
} else {
stepNumCentroids = context.getObj(KMeansTrainBatchOp.CENTROID2);
}
/** 具体业务逻辑代码
* Find the closest cluster for every point and calculate the sums of the points belonging to the same cluster.
*/
}
}
public class KMeansUpdateCentroids extends ComputeFunction {
public void calc(ComContext context) {
if (context.getStepNo() % 2 == 0) {
stepNumCentroids = context.getObj(KMeansTrainBatchOp.CENTROID2);
} else {
stepNumCentroids = context.getObj(KMeansTrainBatchOp.CENTROID1);
}
/** 具体业务逻辑代码
* Update the centroids based on the sum of points and point number belonging to the same cluster.
*/
}
6.4 KMeansOutputModel
这里要特殊说明,因为KMeansOutputModel是最终输出模型,而KMeans算法的实现是:所有subtask都拥有所有中心点,就是说所有subtask都会有相同的模型,就没有必要全部输出,所以这里限定了第一个subtask才能输出,其他的都不输出。
@Override
public List <Row> calc(ComContext context) {
// 只有第一个subtask才输出模型数据。
if (context.getTaskId() != 0) {
return null;
}
....
modelData.params = new KMeansTrainModelData.ParamSummary();
modelData.params.k = k;
modelData.params.vectorColName = vectorColName;
modelData.params.distanceType = distanceType;
modelData.params.vectorSize = vectorSize;
modelData.params.latitudeColName = latitudeColName;
modelData.params.longtitudeColName = longtitudeColName;
RowCollector collector = new RowCollector();
new KMeansModelDataConverter().save(modelData, collector);
return collector.getRows();
}
0xFF 参考
ci.apache.org/projects/fl… 聚类、K-Means、例子、细节
Flink-Gelly:Iterative Graph Processing
Flink 之 Dataflow、Task、subTask、Operator Chains、Slot 介绍
★★★★★★关于生活和技术的思考★★★★★★
微信公众账号:罗西的思考
本文使用 mdnice 排版