Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/12612#discussion_r61658910
  
    --- Diff: core/src/main/scala/org/apache/spark/NewAccumulator.scala ---
    @@ -0,0 +1,391 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark
    +
    +import java.{lang => jl}
    +import java.io.ObjectInputStream
    +import java.util.concurrent.atomic.AtomicLong
    +import javax.annotation.concurrent.GuardedBy
    +
    +import org.apache.spark.scheduler.AccumulableInfo
    +import org.apache.spark.util.Utils
    +
    +
    +private[spark] case class AccumulatorMetadata(
    +    id: Long,
    +    name: Option[String],
    +    countFailedValues: Boolean) extends Serializable
    +
    +
    +/**
    + * The base class for accumulators, that can accumulate inputs of type 
`IN`, and produce output of
    + * type `OUT`.
    + */
    +abstract class NewAccumulator[IN, OUT] extends Serializable {
    +  private[spark] var metadata: AccumulatorMetadata = _
    +  private[this] var atDriverSide = true
    +
    +  private[spark] def register(
    +      sc: SparkContext,
    +      name: Option[String] = None,
    +      countFailedValues: Boolean = false): Unit = {
    +    if (this.metadata != null) {
    +      throw new IllegalStateException("Cannot register an Accumulator 
twice.")
    +    }
    +    this.metadata = AccumulatorMetadata(AccumulatorContext.newId(), name, 
countFailedValues)
    +    AccumulatorContext.register(this)
    +    sc.cleaner.foreach(_.registerAccumulatorForCleanup(this))
    +  }
    +
    +  /**
    +   * Returns true if this accumulator has been registered.  Note that all 
accumulators must be
    +   * registered before ues, or it will throw exception.
    +   */
    +  final def isRegistered: Boolean =
    +    metadata != null && 
AccumulatorContext.originals.containsKey(metadata.id)
    +
    +  private def assertMetadataNotNull(): Unit = {
    +    if (metadata == null) {
    +      throw new IllegalAccessError("The metadata of this accumulator has 
not been assigned yet.")
    +    }
    +  }
    +
    +  /**
    +   * Returns the id of this accumulator, can only be called after 
registration.
    +   */
    +  final def id: Long = {
    +    assertMetadataNotNull()
    +    metadata.id
    +  }
    +
    +  /**
    +   * Returns the name of this accumulator, can only be called after 
registration.
    +   */
    +  final def name: Option[String] = {
    +    assertMetadataNotNull()
    +    metadata.name
    +  }
    +
    +  /**
    +   * Whether to accumulate values from failed tasks. This is set to true 
for system and time
    +   * metrics like serialization time or bytes spilled, and false for 
things with absolute values
    +   * like number of input rows.  This should be used for internal metrics 
only.
    +   */
    +  private[spark] final def countFailedValues: Boolean = {
    +    assertMetadataNotNull()
    +    metadata.countFailedValues
    +  }
    +
    +  /**
    +   * Creates an [[AccumulableInfo]] representation of this 
[[NewAccumulator]] with the provided
    +   * values.
    +   */
    +  private[spark] def toInfo(update: Option[Any], value: Option[Any]): 
AccumulableInfo = {
    +    val isInternal = 
name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX))
    +    new AccumulableInfo(id, name, update, value, isInternal, 
countFailedValues)
    +  }
    +
    +  final private[spark] def isAtDriverSide: Boolean = atDriverSide
    +
    +  /**
    +   * Tells if this accumulator is zero value or not. e.g. for a counter 
accumulator, 0 is zero
    +   * value; for a list accumulator, Nil is zero value.
    +   */
    +  def isZero(): Boolean
    +
    +  /**
    +   * Creates a new copy of this accumulator, which is zero value. i.e. 
call `isZero` on the copy
    +   * must return true.
    +   */
    +  def copyAndReset(): NewAccumulator[IN, OUT]
    +
    +  /**
    +   * Takes the inputs and accumulates. e.g. it can be a simple `+=` for 
counter accumulator.
    +   */
    +  def add(v: IN): Unit
    +
    +  /**
    +   * Merges another same-type accumulator into this one and update its 
state, i.e. this should be
    +   * merge-in-place.
    +   */
    +  def merge(other: NewAccumulator[IN, OUT]): Unit
    +
    +  /**
    +   * Access this accumulator's current value; only allowed on driver.
    +   */
    +  final def value: OUT = {
    +    if (atDriverSide) {
    +      localValue
    +    } else {
    +      throw new UnsupportedOperationException("Can't read accumulator 
value in task")
    +    }
    +  }
    +
    +  /**
    +   * Defines the current value of this accumulator.
    +   *
    +   * This is NOT the global value of the accumulator.  To get the global 
value after a
    +   * completed operation on the dataset, call `value`.
    +   */
    +  def localValue: OUT
    +
    +  // Called by Java when serializing an object
    +  final protected def writeReplace(): Any = {
    +    if (atDriverSide) {
    +      if (!isRegistered) {
    +        throw new UnsupportedOperationException(
    +          "Accumulator must be registered before send to executor")
    +      }
    +      val copy = copyAndReset()
    +      assert(copy.isZero(), "copyAndReset must return a zero value copy")
    +      copy.metadata = metadata
    +      copy
    +    } else {
    +      this
    +    }
    +  }
    +
    +  // Called by Java when deserializing an object
    +  private def readObject(in: ObjectInputStream): Unit = 
Utils.tryOrIOException {
    +    in.defaultReadObject()
    +    if (atDriverSide) {
    +      atDriverSide = false
    +
    +      // Automatically register the accumulator when it is deserialized 
with the task closure.
    +      // This is for external accumulators and internal ones that do not 
represent task level
    +      // metrics, e.g. internal SQL metrics, which are per-operator.
    +      val taskContext = TaskContext.get()
    +      if (taskContext != null) {
    +        taskContext.registerAccumulator(this)
    +      }
    +    } else {
    +      atDriverSide = true
    --- End diff --
    
    When the accumulator is sent back from executor to driver, we should set 
the `atDriverSide` flag.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to