浅谈Flink对象重用(object reuse)

前言

今天是大年初一,祝各位虎年大吉大利~

近期受工作变动影响,博客又荒废了许久。今天难得有空,就前段时间内部技术分享里提到的一个小知识点来写几笔。

对象重用(object reuse)在Flink文档的Execution Configuration一节中并不起眼,并且关于它的说明也语焉不详,如下:

enableObjectReuse() / disableObjectReuse() By default, objects are not reused in Flink. Enabling the object reuse mode will instruct the runtime to reuse user objects for better performance. Keep in mind that this can lead to bugs when the user-code function of an operation is not aware of this behavior.

那么,"reuse"的具体操作是什么?为什么可能会造成bug?什么时候可以安全地启用它呢?本文来简单聊一聊。

算子链与DataStream API对象重用

笔者之前讲过,算子链(operator chaining)是StreamGraph向JobGraph转化过程中的主要优化措施。经过此优化,所有chain在一起的sub-task都会在同一个TaskManager slot中执行,能够减少不必要的数据交换、序列化(注意这点)和上下文切换,从而提高作业的执行效率。

算子链内部的简单示意图如下。

但是,将chained operators连接在一起的ChainingOutput实际上有两种,即ChainingOutputCopyingChainingOutput。查看OperatorChain类中对应的代码:

if (containingTask.getExecutionConfig().isObjectReuseEnabled()) {
    currentOperatorOutput = new ChainingOutput<>(operator, outputTag);
} else {
    TypeSerializer<IN> inSerializer =
            operatorConfig.getTypeSerializerIn1(userCodeClassloader);
    currentOperatorOutput = new CopyingChainingOutput<>(operator, inSerializer, outputTag);
}

也就是说,如果启用了对象重用,构造算子链时采用的是ChainingOutput,反之则是CopyingChainingOutput。它们唯一的不同点就是将StreamRecord推到下游算子时的处理方式,做个对比:

// ChainingOutput#pushToOperator()
protected <X> void pushToOperator(StreamRecord<X> record) {
    try {
        // we know that the given outputTag matches our OutputTag so the record
        // must be of the type that our operator expects.
        @SuppressWarnings("unchecked")
        StreamRecord<T> castRecord = (StreamRecord<T>) record;
        numRecordsIn.inc();
        input.setKeyContextElement(castRecord);
        input.processElement(castRecord);
    } catch (Exception e) {
        throw new ExceptionInChainedOperatorException(e);
    }
}

// CopyingChainingOutput#pushToOperator()
protected <X> void pushToOperator(StreamRecord<X> record) {
    try {
        // we know that the given outputTag matches our OutputTag so the record
        // must be of the type that our operator (and Serializer) expects.
        @SuppressWarnings("unchecked")
        StreamRecord<T> castRecord = (StreamRecord<T>) record;
        numRecordsIn.inc();
        StreamRecord<T> copy = castRecord.copy(serializer.copy(castRecord.getValue()));
        input.setKeyContextElement(copy);
        input.processElement(copy);
    } catch (ClassCastException e) {
        if (outputTag != null) {
            // Enrich error message
            ClassCastException replace =
                    new ClassCastException(
                            String.format(
                                    "%s. Failed to push OutputTag with id '%s' to operator. "
                                            + "This can occur when multiple OutputTags with different types "
                                            + "but identical names are being used.",
                                    e.getMessage(), outputTag.getId()));
            throw new ExceptionInChainedOperatorException(replace);
        } else {
            throw new ExceptionInChainedOperatorException(e);
        }
    } catch (Exception e) {
        throw new ExceptionInChainedOperatorException(e);
    }
}

可见,对象重用的本质就是在算子链中的下游算子使用上游对象的浅拷贝。若关闭对象重用,则必须经过一轮序列化和反序列化,相当于深拷贝,所以就不能100%地发挥算子链的优化效果。

但正所谓鱼与熊掌不可兼得,若启用了对象重用,那么我们的业务代码中必然不能出现以下两种情况,以免造成混乱:

  • 在下游修改上游发射的对象,或者上游存入其State中的对象;
  • 同一条流直接对接多个处理逻辑(如stream.map(new AFunc())的同时还有stream.map(new BFunc()))。

总之,在enableObjectReuse()之前,需要谨慎评估业务代码是否会带来副作用。社区大佬David Anderson曾在Stack Overflow上给出了一个简单明晰的回答,可参见这里

Flink SQL中的对象重用

另一位社区大佬Nico Kruber曾经写过一篇名为<<A Journey to Beating Flink's SQL Performance>>的文章,其中说启用对象重用可以为Blink Planner带来可观的性能收益,并且还相当安全。为什么?

我们知道,Flink SQL的类型系统与DataStream Runtime原生的类型系统有一定区别,故某些基础数据类型的序列化器的实现也有不同。以最常见的字符串类型为例,DataStream原生的StringSerializercopy()方法如下。

@Override
public String copy(String from) {
    return from;
}

可见是能够利用String类型本身的不可变性(immutability)来避免真正的复制。所以,若DataStream API程序中的复杂数据类型越少,序列化成本就越低,打开对象重用的收益也就越小。前述的文章也说明了这一点。

Flink SQL体系中的StringDataSerializer#copy()方法则完全不一样,如下(实际上是BinaryStringData#copy())。

public BinaryStringData copy() {
    ensureMaterialized();
    byte[] copy =
            BinarySegmentUtils.copyToBytes(
                    binarySection.segments, binarySection.offset, binarySection.sizeInBytes);
    return new BinaryStringData(
            new MemorySegment[] {MemorySegmentFactory.wrap(copy)},
            0,
            binarySection.sizeInBytes,
            javaObject);
}

可见是要实打实地复制底层的MemorySegment,此时对象重用的优点就很明显了。

如何保证这边不会有像DataStream API同样的隐患?答案在(之前讲过的)代码生成阶段。例如,在查询维表的CommonExecLookupJoin执行节点中,生成访问输入字段的代码时,会判断是否要强制深拷贝(当允许对象重用时,deepCopy就为true):

  def generateFieldAccess(
    ctx: CodeGeneratorContext,
    inputType: LogicalType,
    inputTerm: String,
    index: Int,
    deepCopy: Boolean): GeneratedExpression = {
    val expr = generateFieldAccess(ctx, inputType, inputTerm, index)
    if (deepCopy) {    // 
      expr.deepCopy(ctx)
    } else {
      expr
    }
  }

如果结果类型是可变(mutable)类型的话,就会生成新的拷贝代码,防止出问题。

def deepCopy(ctx: CodeGeneratorContext): GeneratedExpression = {
  // only copy when type is mutable
  if (TypeCheckUtils.isMutable(resultType)) {
    // if the type need copy, it must be a boxed type
    val typeTerm = boxedTypeTermForType(resultType)
    val serTerm = ctx.addReusableTypeSerializer(resultType)
    val newResultTerm = ctx.addReusableLocalVariable(typeTerm, "field")
    val newCode =
      s"""
         |$code
         |$newResultTerm = $resultTerm;
         |if (!$nullTerm) {
         |  $newResultTerm = ($typeTerm) ($serTerm.copy($newResultTerm));
         |}
      """.stripMargin
    GeneratedExpression(newResultTerm, nullTerm, newCode, resultType, literalValue)
  } else {
    this
  }
}

The End

边看《开端》边写的这一篇,三心二意,有错误请批评指正(

京东物流人工智能与大数据部持续招人中,各位有意年后换工作的大佬尽管丢简历过来,JDL欢迎你~

民那晚安(

發表評論
所有評論
還沒有人評論,想成為第一個評論的人麼? 請在上方評論欄輸入並且點擊發布.
相關文章