Github user MLnick commented on a diff in the pull request: https://github.com/apache/spark/pull/12612#discussion_r61414486 --- 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") --- End diff -- @cloud-fan I'm getting intermittent, but regular, test failures in `ALSSuite` (not sure if there might be others, this just happens to be something I'm working on now). e.g. ``` [info] - exact rank-1 matrix *** FAILED *** (4 seconds, 397 milliseconds) [info] org.apache.spark.SparkException: Job aborted due to stage failure: Failed to serialize task 74, not attempting to retry it. Exception during serialization: java.lang.UnsupportedOperationException: Accumulator must be registered before send to executor [info] at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1448) [info] at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1436) [info] at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1435) [info] at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) [info] at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) [info] at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1435) [info] at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) [info] at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) [info] at scala.Option.foreach(Option.scala:257) [info] at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:809) [info] at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1657) [info] at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1616) [info] at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1605) [info] at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) [info] at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:630) [info] at org.apache.spark.SparkContext.runJob(SparkContext.scala:1873) [info] at org.apache.spark.SparkContext.runJob(SparkContext.scala:1936) [info] at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:970) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) [info] at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) [info] at org.apache.spark.rdd.RDD.reduce(RDD.scala:952) [info] at org.apache.spark.rdd.DoubleRDDFunctions$$anonfun$stats$1.apply(DoubleRDDFunctions.scala:42) [info] at org.apache.spark.rdd.DoubleRDDFunctions$$anonfun$stats$1.apply(DoubleRDDFunctions.scala:42) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) [info] at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) [info] at org.apache.spark.rdd.DoubleRDDFunctions.stats(DoubleRDDFunctions.scala:41) [info] at org.apache.spark.rdd.DoubleRDDFunctions$$anonfun$mean$1.apply$mcD$sp(DoubleRDDFunctions.scala:47) [info] at org.apache.spark.rdd.DoubleRDDFunctions$$anonfun$mean$1.apply(DoubleRDDFunctions.scala:47) [info] at org.apache.spark.rdd.DoubleRDDFunctions$$anonfun$mean$1.apply(DoubleRDDFunctions.scala:47) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) [info] at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) [info] at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) [info] at org.apache.spark.rdd.DoubleRDDFunctions.mean(DoubleRDDFunctions.scala:46) [info] at org.apache.spark.ml.recommendation.ALSSuite.testALS(ALSSuite.scala:373) [info] at org.apache.spark.ml.recommendation.ALSSuite$$anonfun$12.apply$mcV$sp(ALSSuite.scala:385) [info] at org.apache.spark.ml.recommendation.ALSSuite$$anonfun$12.apply(ALSSuite.scala:383) [info] at org.apache.spark.ml.recommendation.ALSSuite$$anonfun$12.apply(ALSSuite.scala:383) [info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) [info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:56) [info] at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuite.runTest(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) [info] at scala.collection.immutable.List.foreach(List.scala:381) [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) [info] at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) [info] at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) [info] at org.scalatest.Suite$class.run(Suite.scala:1424) [info] at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:545) [info] at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) [info] at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:28) [info] at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) [info] at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) [info] at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:28) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:357) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:502) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:296) [info] at sbt.ForkMain$Run$2.call(ForkMain.java:286) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [info] at java.lang.Thread.run(Thread.java:745) ```
--- 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