• SPARK中的wholeStageCodegen全代码生成--以aggregate代码生成为例说起(2)


    背景

    本文基于 SPARK 3.3.0
    从一个unit test来探究SPARK Codegen的逻辑,

      test("SortAggregate should be included in WholeStageCodegen") {
        val df = spark.range(10).agg(max(col("id")), avg(col("id")))
        withSQLConf("spark.sql.test.forceApplySortAggregate" -> "true") {
          val plan = df.queryExecution.executedPlan
          assert(plan.exists(p =>
            p.isInstanceOf[WholeStageCodegenExec] &&
              p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortAggregateExec]))
          assert(df.collect() === Array(Row(9, 4.5)))
        }
      }
    该sql形成的执行计划第一部分的全代码生成部分如下:
    
    • 1
    • 2
    • 3
    • 4
    • 5
    • 6
    • 7
    • 8
    • 9
    • 10
    • 11
    WholeStageCodegen
    
    • 1

    ± *(1) SortAggregate(key=[], functions=[partial_max(id#0L), partial_avg(id#0L)], output=[max#12L, sum#13, count#14L])
    ± *(1) Range (0, 10, step=1, splits=2)

    
    
    • 1

    分析

    第一阶段wholeStageCodegen

    第一阶段的代码生成涉及到SortAggregateExec和RangeExec的produce和consume方法,这里一一来分析:
    第一阶段wholeStageCodegen数据流如下:

     WholeStageCodegenExec      SortAggregateExec(partial)     RangeExec        
      =========================================================================
     
      -> execute()
          |
       doExecute() --------->   inputRDDs() -----------------> inputRDDs() 
          |
       doCodeGen()
          |
          +----------------->   produce()
                                  |
                               doProduce() 
                                  |
                               doProduceWithoutKeys() -------> produce()
                                                                  |
                                                              doProduce()
                                                                  |
                               doConsume()<------------------- consume()
                                  |
                               doConsumeWithoutKeys()
                                  |并不是doConsumeWithoutKeys调用consume,而是由doProduceWithoutKeys调用
       doConsume()  <--------  consume()
    
    
    • 1
    • 2
    • 3
    • 4
    • 5
    • 6
    • 7
    • 8
    • 9
    • 10
    • 11
    • 12
    • 13
    • 14
    • 15
    • 16
    • 17
    • 18
    • 19
    • 20
    • 21
    • 22
    • 23
    sortAggregateExec(partial)的produce

    doProduce最终调用方法doProduceWithoutKeys,该部分代码如下:

    private def doProduceWithoutKeys(ctx: CodegenContext): String = {
        val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg")
        // The generated function doesn't have input row in the code context.
        ctx.INPUT_ROW = null
    
        // generate variables for aggregation buffer
        val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
        val initExpr = functions.map(f => f.initialValues)
        bufVars = initExpr.map { exprs =>
          exprs.map { e =>
            val isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "bufIsNull")
            val value = ctx.addMutableState(CodeGenerator.javaType(e.dataType), "bufValue")
            // The initial expression should not access any column
            val ev = e.genCode(ctx)
            val initVars =
              code"""
                    |$isNull = ${ev.isNull};
                    |$value = ${ev.value};
                  """.stripMargin
            ExprCode(
              ev.code + initVars,
              JavaCode.isNullGlobal(isNull),
              JavaCode.global(value, e.dataType))
          }
        }
        val flatBufVars = bufVars.flatten
        val initBufVar = evaluateVariables(flatBufVars)
    
        // generate variables for output
        val (resultVars, genResult) = if (modes.contains(Final) || modes.contains(Complete)) {
          // evaluate aggregate results
          ctx.currentVars = flatBufVars
          val aggResults = bindReferences(
            functions.map(_.evaluateExpression),
            aggregateBufferAttributes).map(_.genCode(ctx))
          println(s"aggResults: ${aggResults}")
          val evaluateAggResults = evaluateVariables(aggResults)
          // evaluate result expressions
          ctx.currentVars = aggResults
          val resultVars = bindReferences(resultExpressions, aggregateAttributes).map(_.genCode(ctx))
          (resultVars,
            s"""
               |$evaluateAggResults
               |${evaluateVariables(resultVars)}
             """.stripMargin)
        } else if (modes.contains(Partial) || modes.contains(PartialMerge)) {
          // output the aggregate buffer directly
          (flatBufVars, "")
        } else {
          // no aggregate function, the result should be literals
          val resultVars = resultExpressions.map(_.genCode(ctx))
          (resultVars, evaluateVariables(resultVars))
        }
    
        val doAgg = ctx.freshName("doAggregateWithoutKey")
        val doAggFuncName = ctx.addNewFunction(doAgg,
          s"""
             |private void $doAgg() throws java.io.IOException {
             |  // initialize aggregation buffer
             |  $initBufVar
             |
             |  ${child.asInstanceOf[CodegenSupport].produce(ctx, this)}
             |}
           """.stripMargin)
    
        val numOutput = metricTerm(ctx, "numOutputRows")
        val doAggWithRecordMetric =
          if (needHashTable) {
            val aggTime = metricTerm(ctx, "aggTime")
            val beforeAgg = ctx.freshName("beforeAgg")
            s"""
               |long $beforeAgg = System.nanoTime();
               |$doAggFuncName();
               |$aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS);
             """.stripMargin
          } else {
            s"$doAggFuncName();"
          }
    
        s"""
           |while (!$initAgg) {
           |  $initAgg = true;
           |  $doAggWithRecordMetric
           |
           |  // output the result
           |  ${genResult.trim}
           |
           |  $numOutput.add(1);
           |  ${consume(ctx, resultVars).trim}
           |}
         """.stripMargin
      }
    
    • 1
    • 2
    • 3
    • 4
    • 5
    • 6
    • 7
    • 8
    • 9
    • 10
    • 11
    • 12
    • 13
    • 14
    • 15
    • 16
    • 17
    • 18
    • 19
    • 20
    • 21
    • 22
    • 23
    • 24
    • 25
    • 26
    • 27
    • 28
    • 29
    • 30
    • 31
    • 32
    • 33
    • 34
    • 35
    • 36
    • 37
    • 38
    • 39
    • 40
    • 41
    • 42
    • 43
    • 44
    • 45
    • 46
    • 47
    • 48
    • 49
    • 50
    • 51
    • 52
    • 53
    • 54
    • 55
    • 56
    • 57
    • 58
    • 59
    • 60
    • 61
    • 62
    • 63
    • 64
    • 65
    • 66
    • 67
    • 68
    • 69
    • 70
    • 71
    • 72
    • 73
    • 74
    • 75
    • 76
    • 77
    • 78
    • 79
    • 80
    • 81
    • 82
    • 83
    • 84
    • 85
    • 86
    • 87
    • 88
    • 89
    • 90
    • 91
    • 92
    • val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, “initAgg”)
      用来进行初始化聚合的判断,便于只进行一次代码生成

    • ctx.INPUT_ROW = null
      这里把INPUT_ROW设置为null的原因是来判断BoundReference绑定的值是否来自于InternalRow类型的变量,这样的话,就得调用InternalRow对应的方法获取对应的值,如getLong方法。
      这里设置为null,说明不是来自于InternalRow类型的变量(也就是计算的值大概率不是来自于其他算子的计算结果),也就是直接赋值。

    • val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
      对于这一句为什么 aggregateFunction一定是DeclarativeAggregate类型呢?为什么不是ImperativeAggregate类型的呢?
      其实因为是ImperativeAggregate是继承自CodegenFallback的,这在CollapseCodegenStages规则中supportCodegen方法中就会进行判断不符合全代码生成的条件,自然就不会有代码生成这一步,所以aggregateFunction只能是DeclarativeAggregate类型的。

    • val initExpr = functions.map(f => f.initialValues)
      这个是聚合函数的初始值,对于avg来说则是 Seq( /* sum = */ Literal.default(sumDataType),/* count = */ Literal(0L)) ,如没特别说明,我们就只讲解AVG的代码生成部分,因为MAX等表达式原理是一样的。(AVG则是由SUM和COUNT两个缓冲值组成)

    • bufVars = …
      这是一个赋值操作,其中ctx.addMutableState()操作则是声明变量,这里的变量属于全局变量,也就是类的成员变量,前缀是当前类的前缀,具体是在CodegenSupport
      variablePrefix方法中,对于SortAggregateExec则对应为sortAgg,通过该方法会在对应的生成类中,生成如下的成员变量:

       //对应于sum
       private boolean sortAgg_bufIsNull_0;
       private long sortAgg_bufValue_0;
       //对应于count
       private boolean sortAgg_bufIsNull_1; 
       private long sortAgg_bufValue_1;
      
      • 1
      • 2
      • 3
      • 4
      • 5
      • 6

      initVars=这部分则是根据聚合函数的初始值的代码生成部分,初始化成员变量sortAgg_bufIsNull_0,sortAgg_bufValue_0等值,具体的初始化的部分是在下面
      其中为什么有IsNull参数?是因为如果说该参数为NULL的话,代码生成的时候就得去判断是否为null,否则就会出现异常。

    • initBufVar=
      这部分代码是上面提到的初始化类的成员变量,具体在哪里初始化呢? 是在聚合函数的一开始,如下:

        private void sortAgg_doAggregateWithoutKey_0() throws java.io.IOException {
         // initialize aggregation buffer
         sortAgg_bufIsNull_0 = true;
         sortAgg_bufValue_0 = -1L; 
      
      • 1
      • 2
      • 3
      • 4
    • val (resultVars, genResult) =
      这部分会根据是部分聚合(Partial)还是最终的聚合(Final)来进行分之判断:
      所有对应到Partial则是 (flatBufVars, “”),所以这部分直接把SUM和COUNT(属于AVG的计算缓存)赋值给了resultVars, genResult则是为空,因为不需要做任何处理。

    • val doAggFuncName =
      这部分调用RangExec的produce方法生成代码,而且对于initBufVar的初始化代码也在这里。

    • val numOutput = metricTerm(ctx, “numOutputRows”)和val doAggWithRecordMetric =
      这里会调用metricTerm方法,从而创建指标,这些指标变量会以方法参数形式传递给*WholeStageCodegenExec中的clazz.generate(references).*方法

    • 组装代码
      最后一步:*while (!$initAgg) * 是组装代码
      doAggWithRecordMetric 是调用child.produce.
      genResult.trim 因为这里是Partial Aggregate,所以为空.
      numOutput.add(1) 是对输出的记录数加一
      consume(ctx, resultVars).trirm 是对输出的数据进行组装,组装成UnsafeRow以便spark进行的后续处理,也就是在此以后返回的数据就是正常的InteralRow的处理方式了,对于consume()这部分代码我们后续再说,在这里我们先按照数据流的方式来解释代码。

    第一阶段wholeStageCodegen生成的代码

    第一阶段wholeStageCodegen生成的代码如下:

    /* 001 */ public Object generate(Object[] references) {
    /* 002 */   return new GeneratedIteratorForCodegenStage1(references);
    /* 003 */ }
    /* 004 */
    /* 005 */ // codegenStageId=1
    /* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
    /* 007 */   private Object[] references;
    /* 008 */   private scala.collection.Iterator[] inputs;
    /* 009 */   private boolean sortAgg_initAgg_0;
    /* 010 */   private boolean sortAgg_bufIsNull_0;
    /* 011 */   private long sortAgg_bufValue_0;
    /* 012 */   private boolean sortAgg_bufIsNull_1;
    /* 013 */   private double sortAgg_bufValue_1;
    /* 014 */   private boolean sortAgg_bufIsNull_2;
    /* 015 */   private long sortAgg_bufValue_2;
    /* 016 */   private boolean range_initRange_0;
    /* 017 */   private long range_nextIndex_0;
    /* 018 */   private TaskContext range_taskContext_0;
    /* 019 */   private InputMetrics range_inputMetrics_0;
    /* 020 */   private long range_batchEnd_0;
    /* 021 */   private long range_numElementsTodo_0;
    /* 022 */   private boolean sortAgg_sortAgg_isNull_4_0;
    /* 023 */   private boolean sortAgg_sortAgg_isNull_9_0;
    /* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[] range_mutableStateArray_0 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter[3];
    /* 025 */
    /* 026 */   public GeneratedIteratorForCodegenStage1(Object[] references) {
    /* 027 */     this.references = references;
    /* 028 */   }
    /* 029 */
    /* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
    /* 031 */     partitionIndex = index;
    /* 032 */     this.inputs = inputs;
    /* 033 */
    /* 034 */     range_taskContext_0 = TaskContext.get();
    /* 035 */     range_inputMetrics_0 = range_taskContext_0.taskMetrics().inputMetrics();
    /* 036 */     range_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
    /* 037 */     range_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(1, 0);
    /* 038 */     range_mutableStateArray_0[2] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(3, 0);
    /* 039 */
    /* 040 */   }
    /* 041 */
    /* 042 */   private void sortAgg_doAggregate_max_0(long sortAgg_expr_0_0) throws java.io.IOException {
    /* 043 */     sortAgg_sortAgg_isNull_4_0 = true;
    /* 044 */     long sortAgg_value_4 = -1L;
    /* 045 */
    /* 046 */     if (!sortAgg_bufIsNull_0 && (sortAgg_sortAgg_isNull_4_0 ||
    /* 047 */         sortAgg_bufValue_0 > sortAgg_value_4)) {
    /* 048 */       sortAgg_sortAgg_isNull_4_0 = false;
    /* 049 */       sortAgg_value_4 = sortAgg_bufValue_0;
    /* 050 */     }
    /* 051 */
    /* 052 */     if (!false && (sortAgg_sortAgg_isNull_4_0 ||
    /* 053 */         sortAgg_expr_0_0 > sortAgg_value_4)) {
    /* 054 */       sortAgg_sortAgg_isNull_4_0 = false;
    /* 055 */       sortAgg_value_4 = sortAgg_expr_0_0;
    /* 056 */     }
    /* 057 */
    /* 058 */     sortAgg_bufIsNull_0 = sortAgg_sortAgg_isNull_4_0;
    /* 059 */     sortAgg_bufValue_0 = sortAgg_value_4;
    /* 060 */   }
    /* 061 */
    /* 062 */   private void sortAgg_doAggregateWithoutKey_0() throws java.io.IOException {
    /* 063 */     // initialize aggregation buffer
    /* 064 */     sortAgg_bufIsNull_0 = true;
    /* 065 */     sortAgg_bufValue_0 = -1L;
    /* 066 */     sortAgg_bufIsNull_1 = false;
    /* 067 */     sortAgg_bufValue_1 = 0.0D;
    /* 068 */     sortAgg_bufIsNull_2 = false;
    /* 069 */     sortAgg_bufValue_2 = 0L;
    /* 070 */
    /* 071 */     // initialize Range
    /* 072 */     if (!range_initRange_0) {
    /* 073 */       range_initRange_0 = true;
    /* 074 */       initRange(partitionIndex);
    /* 075 */     }
    /* 076 */
    /* 077 */     while (true) {
    /* 078 */       if (range_nextIndex_0 == range_batchEnd_0) {
    /* 079 */         long range_nextBatchTodo_0;
    /* 080 */         if (range_numElementsTodo_0 > 1000L) {
    /* 081 */           range_nextBatchTodo_0 = 1000L;
    /* 082 */           range_numElementsTodo_0 -= 1000L;
    /* 083 */         } else {
    /* 084 */           range_nextBatchTodo_0 = range_numElementsTodo_0;
    /* 085 */           range_numElementsTodo_0 = 0;
    /* 086 */           if (range_nextBatchTodo_0 == 0) break;
    /* 087 */         }
    /* 088 */         range_batchEnd_0 += range_nextBatchTodo_0 * 1L;
    /* 089 */       }
    /* 090 */
    /* 091 */       int range_localEnd_0 = (int)((range_batchEnd_0 - range_nextIndex_0) / 1L);
    /* 092 */       for (int range_localIdx_0 = 0; range_localIdx_0 < range_localEnd_0; range_localIdx_0++) {
    /* 093 */         long range_value_0 = ((long)range_localIdx_0 * 1L) + range_nextIndex_0;
    /* 094 */
    /* 095 */         sortAgg_doConsume_0(range_value_0);
    /* 096 */
    /* 097 */         // shouldStop check is eliminated
    /* 098 */       }
    /* 099 */       range_nextIndex_0 = range_batchEnd_0;
    /* 100 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(range_localEnd_0);
    /* 101 */       range_inputMetrics_0.incRecordsRead(range_localEnd_0);
    /* 102 */       range_taskContext_0.killTaskIfInterrupted();
    /* 103 */     }
    /* 104 */
    /* 105 */   }
    /* 106 */
    /* 107 */   private void initRange(int idx) {
    /* 108 */     java.math.BigInteger index = java.math.BigInteger.valueOf(idx);
    /* 109 */     java.math.BigInteger numSlice = java.math.BigInteger.valueOf(2L);
    /* 110 */     java.math.BigInteger numElement = java.math.BigInteger.valueOf(10L);
    /* 111 */     java.math.BigInteger step = java.math.BigInteger.valueOf(1L);
    /* 112 */     java.math.BigInteger start = java.math.BigInteger.valueOf(0L);
    /* 113 */     long partitionEnd;
    /* 114 */
    /* 115 */     java.math.BigInteger st = index.multiply(numElement).divide(numSlice).multiply(step).add(start);
    /* 116 */     if (st.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
    /* 117 */       range_nextIndex_0 = Long.MAX_VALUE;
    /* 118 */     } else if (st.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
    /* 119 */       range_nextIndex_0 = Long.MIN_VALUE;
    /* 120 */     } else {
    /* 121 */       range_nextIndex_0 = st.longValue();
    /* 122 */     }
    /* 123 */     range_batchEnd_0 = range_nextIndex_0;
    /* 124 */
    /* 125 */     java.math.BigInteger end = index.add(java.math.BigInteger.ONE).multiply(numElement).divide(numSlice)
    /* 126 */     .multiply(step).add(start);
    /* 127 */     if (end.compareTo(java.math.BigInteger.valueOf(Long.MAX_VALUE)) > 0) {
    /* 128 */       partitionEnd = Long.MAX_VALUE;
    /* 129 */     } else if (end.compareTo(java.math.BigInteger.valueOf(Long.MIN_VALUE)) < 0) {
    /* 130 */       partitionEnd = Long.MIN_VALUE;
    /* 131 */     } else {
    /* 132 */       partitionEnd = end.longValue();
    /* 133 */     }
    /* 134 */
    /* 135 */     java.math.BigInteger startToEnd = java.math.BigInteger.valueOf(partitionEnd).subtract(
    /* 136 */       java.math.BigInteger.valueOf(range_nextIndex_0));
    /* 137 */     range_numElementsTodo_0  = startToEnd.divide(step).longValue();
    /* 138 */     if (range_numElementsTodo_0 < 0) {
    /* 139 */       range_numElementsTodo_0 = 0;
    /* 140 */     } else if (startToEnd.remainder(step).compareTo(java.math.BigInteger.valueOf(0L)) != 0) {
    /* 141 */       range_numElementsTodo_0++;
    /* 142 */     }
    /* 143 */   }
    /* 144 */
    /* 145 */   protected void processNext() throws java.io.IOException {
    /* 146 */     while (!sortAgg_initAgg_0) {
    /* 147 */       sortAgg_initAgg_0 = true;
    /* 148 */       sortAgg_doAggregateWithoutKey_0();
    /* 149 */
    /* 150 */       // output the result
    /* 151 */
    /* 152 */       ((org.apache.spark.sql.execution.metric.SQLMetric) references[1] /* numOutputRows */).add(1);
    /* 153 */       range_mutableStateArray_0[2].reset();
    /* 154 */
    /* 155 */       range_mutableStateArray_0[2].zeroOutNullBytes();
    /* 156 */
    /* 157 */       if (sortAgg_bufIsNull_0) {
    /* 158 */         range_mutableStateArray_0[2].setNullAt(0);
    /* 159 */       } else {
    /* 160 */         range_mutableStateArray_0[2].write(0, sortAgg_bufValue_0);
    /* 161 */       }
    /* 162 */
    /* 163 */       if (sortAgg_bufIsNull_1) {
    /* 164 */         range_mutableStateArray_0[2].setNullAt(1);
    /* 165 */       } else {
    /* 166 */         range_mutableStateArray_0[2].write(1, sortAgg_bufValue_1);
    /* 167 */       }
    /* 168 */
    /* 169 */       if (sortAgg_bufIsNull_2) {
    /* 170 */         range_mutableStateArray_0[2].setNullAt(2);
    /* 171 */       } else {
    /* 172 */         range_mutableStateArray_0[2].write(2, sortAgg_bufValue_2);
    /* 173 */       }
    /* 174 */       append((range_mutableStateArray_0[2].getRow()));
    /* 175 */     }
    /* 176 */   }
    /* 177 */
    /* 178 */   private void sortAgg_doConsume_0(long sortAgg_expr_0_0) throws java.io.IOException {
    /* 179 */     // do aggregate
    /* 180 */     // common sub-expressions
    /* 181 */
    /* 182 */     // evaluate aggregate functions and update aggregation buffers
    /* 183 */     sortAgg_doAggregate_max_0(sortAgg_expr_0_0);
    /* 184 */     sortAgg_doAggregate_avg_0(sortAgg_expr_0_0);
    /* 185 */
    /* 186 */   }
    /* 187 */
    /* 188 */   private void sortAgg_doAggregate_avg_0(long sortAgg_expr_0_0) throws java.io.IOException {
    /* 189 */     boolean sortAgg_isNull_7 = true;
    /* 190 */     double sortAgg_value_7 = -1.0;
    /* 191 */
    /* 192 */     if (!sortAgg_bufIsNull_1) {
    /* 193 */       sortAgg_sortAgg_isNull_9_0 = true;
    /* 194 */       double sortAgg_value_9 = -1.0;
    /* 195 */       do {
    /* 196 */         boolean sortAgg_isNull_10 = false;
    /* 197 */         double sortAgg_value_10 = -1.0;
    /* 198 */         if (!false) {
    /* 199 */           sortAgg_value_10 = (double) sortAgg_expr_0_0;
    /* 200 */         }
    /* 201 */         if (!sortAgg_isNull_10) {
    /* 202 */           sortAgg_sortAgg_isNull_9_0 = false;
    /* 203 */           sortAgg_value_9 = sortAgg_value_10;
    /* 204 */           continue;
    /* 205 */         }
    /* 206 */
    /* 207 */         if (!false) {
    /* 208 */           sortAgg_sortAgg_isNull_9_0 = false;
    /* 209 */           sortAgg_value_9 = 0.0D;
    /* 210 */           continue;
    /* 211 */         }
    /* 212 */
    /* 213 */       } while (false);
    /* 214 */
    /* 215 */       sortAgg_isNull_7 = false; // resultCode could change nullability.
    /* 216 */
    /* 217 */       sortAgg_value_7 = sortAgg_bufValue_1 + sortAgg_value_9;
    /* 218 */
    /* 219 */     }
    /* 220 */     boolean sortAgg_isNull_13 = false;
    /* 221 */     long sortAgg_value_13 = -1L;
    /* 222 */     if (!false && false) {
    /* 223 */       sortAgg_isNull_13 = sortAgg_bufIsNull_2;
    /* 224 */       sortAgg_value_13 = sortAgg_bufValue_2;
    /* 225 */     } else {
    /* 226 */       boolean sortAgg_isNull_17 = true;
    /* 227 */       long sortAgg_value_17 = -1L;
    /* 228 */
    /* 229 */       if (!sortAgg_bufIsNull_2) {
    /* 230 */         sortAgg_isNull_17 = false; // resultCode could change nullability.
    /* 231 */
    /* 232 */         sortAgg_value_17 = sortAgg_bufValue_2 + 1L;
    /* 233 */
    /* 234 */       }
    /* 235 */       sortAgg_isNull_13 = sortAgg_isNull_17;
    /* 236 */       sortAgg_value_13 = sortAgg_value_17;
    /* 237 */     }
    /* 238 */
    /* 239 */     sortAgg_bufIsNull_1 = sortAgg_isNull_7;
    /* 240 */     sortAgg_bufValue_1 = sortAgg_value_7;
    /* 241 */
    /* 242 */     sortAgg_bufIsNull_2 = sortAgg_isNull_13;
    /* 243 */     sortAgg_bufValue_2 = sortAgg_value_13;
    /* 244 */   }
    /* 245 */
    /* 246 */ }
    
    • 1
    • 2
    • 3
    • 4
    • 5
    • 6
    • 7
    • 8
    • 9
    • 10
    • 11
    • 12
    • 13
    • 14
    • 15
    • 16
    • 17
    • 18
    • 19
    • 20
    • 21
    • 22
    • 23
    • 24
    • 25
    • 26
    • 27
    • 28
    • 29
    • 30
    • 31
    • 32
    • 33
    • 34
    • 35
    • 36
    • 37
    • 38
    • 39
    • 40
    • 41
    • 42
    • 43
    • 44
    • 45
    • 46
    • 47
    • 48
    • 49
    • 50
    • 51
    • 52
    • 53
    • 54
    • 55
    • 56
    • 57
    • 58
    • 59
    • 60
    • 61
    • 62
    • 63
    • 64
    • 65
    • 66
    • 67
    • 68
    • 69
    • 70
    • 71
    • 72
    • 73
    • 74
    • 75
    • 76
    • 77
    • 78
    • 79
    • 80
    • 81
    • 82
    • 83
    • 84
    • 85
    • 86
    • 87
    • 88
    • 89
    • 90
    • 91
    • 92
    • 93
    • 94
    • 95
    • 96
    • 97
    • 98
    • 99
    • 100
    • 101
    • 102
    • 103
    • 104
    • 105
    • 106
    • 107
    • 108
    • 109
    • 110
    • 111
    • 112
    • 113
    • 114
    • 115
    • 116
    • 117
    • 118
    • 119
    • 120
    • 121
    • 122
    • 123
    • 124
    • 125
    • 126
    • 127
    • 128
    • 129
    • 130
    • 131
    • 132
    • 133
    • 134
    • 135
    • 136
    • 137
    • 138
    • 139
    • 140
    • 141
    • 142
    • 143
    • 144
    • 145
    • 146
    • 147
    • 148
    • 149
    • 150
    • 151
    • 152
    • 153
    • 154
    • 155
    • 156
    • 157
    • 158
    • 159
    • 160
    • 161
    • 162
    • 163
    • 164
    • 165
    • 166
    • 167
    • 168
    • 169
    • 170
    • 171
    • 172
    • 173
    • 174
    • 175
    • 176
    • 177
    • 178
    • 179
    • 180
    • 181
    • 182
    • 183
    • 184
    • 185
    • 186
    • 187
    • 188
    • 189
    • 190
    • 191
    • 192
    • 193
    • 194
    • 195
    • 196
    • 197
    • 198
    • 199
    • 200
    • 201
    • 202
    • 203
    • 204
    • 205
    • 206
    • 207
    • 208
    • 209
    • 210
    • 211
    • 212
    • 213
    • 214
    • 215
    • 216
    • 217
    • 218
    • 219
    • 220
    • 221
    • 222
    • 223
    • 224
    • 225
    • 226
    • 227
    • 228
    • 229
    • 230
    • 231
    • 232
    • 233
    • 234
    • 235
    • 236
    • 237
    • 238
    • 239
    • 240
    • 241
    • 242
    • 243
    • 244
    • 245
    • 246
  • 相关阅读:
    Packet Tracer 实验 - 排除多区域 OSPFv3 故障
    linux常用命令-1
    含文档+PPT+源码等]精品springboot+VUE的学生宿舍管理系统设计与实现Java项目源码
    【数据结构】—带头双向循环链表的实现(完美链表)
    springboot闲置衣物捐赠系统毕业设计源码021009
    ASEMI整流桥KBL406参数,KBL406规格,KBL406封装
    NSSM部署window服务
    slate源码解析(一)- 序言
    前端Vue的循环forEach等各种循环下,取每一项下面的有个对象里面的值方法
    千钧一发之际赢得暂缓令,苹果App Store要反败为胜了?
  • 原文地址:https://blog.csdn.net/monkeyboy_tech/article/details/126754907