/
AccumulatorV2.scala
506 lines (423 loc) · 14.8 KB
/
AccumulatorV2.scala
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
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
/*
* 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.util
import java.{lang => jl}
import java.io.ObjectInputStream
import java.util.{ArrayList, Collections}
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicLong
import scala.collection.JavaConverters._
import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext}
import org.apache.spark.scheduler.AccumulableInfo
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`.
*
* `OUT` should be a type that can be read atomically (e.g., Int, Long), or thread-safely
* (e.g., synchronized collections) because it will be read from other threads.
*/
abstract class AccumulatorV2[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 All accumulators must be registered before use, or it will throw exception.
*/
final def isRegistered: Boolean =
metadata != null && AccumulatorContext.get(metadata.id).isDefined
private def assertMetadataNotNull(): Unit = {
if (metadata == null) {
throw new IllegalStateException("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()
if (atDriverSide) {
metadata.name.orElse(AccumulatorContext.get(id).flatMap(_.metadata.name))
} else {
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 [[AccumulatorV2]] 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
/**
* Returns 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(): AccumulatorV2[IN, OUT] = {
val copyAcc = copy()
copyAcc.reset()
copyAcc
}
/**
* Creates a new copy of this accumulator.
*/
def copy(): AccumulatorV2[IN, OUT]
/**
* Resets this accumulator, which is zero value. i.e. call `isZero` must
* return true.
*/
def reset(): Unit
/**
* Takes the inputs and accumulates.
*/
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: AccumulatorV2[IN, OUT]): Unit
/**
* Defines the current value of this accumulator
*/
def value: 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 copyAcc = copyAndReset()
assert(copyAcc.isZero, "copyAndReset must return a zero value copy")
val isInternalAcc = name.isDefined && name.get.startsWith(InternalAccumulator.METRICS_PREFIX)
if (isInternalAcc) {
// Do not serialize the name of internal accumulator and send it to executor.
copyAcc.metadata = metadata.copy(name = None)
} else {
// For non-internal accumulators, we still need to send the name because users may need to
// access the accumulator name at executor side, or they may keep the accumulators sent from
// executors and access the name when the registered accumulator is already garbage
// collected(e.g. SQLMetrics).
copyAcc.metadata = metadata
}
copyAcc
} 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
}
}
override def toString: String = {
if (metadata == null) {
"Un-registered Accumulator: " + getClass.getSimpleName
} else {
getClass.getSimpleName + s"(id: $id, name: $name, value: $value)"
}
}
}
/**
* An internal class used to track accumulators by Spark itself.
*/
private[spark] object AccumulatorContext {
/**
* This global map holds the original accumulator objects that are created on the driver.
* It keeps weak references to these objects so that accumulators can be garbage-collected
* once the RDDs and user-code that reference them are cleaned up.
* TODO: Don't use a global map; these should be tied to a SparkContext (SPARK-13051).
*/
private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[AccumulatorV2[_, _]]]
private[this] val nextId = new AtomicLong(0L)
/**
* Returns a globally unique ID for a new [[AccumulatorV2]].
* Note: Once you copy the [[AccumulatorV2]] the ID is no longer unique.
*/
def newId(): Long = nextId.getAndIncrement
/** Returns the number of accumulators registered. Used in testing. */
def numAccums: Int = originals.size
/**
* Registers an [[AccumulatorV2]] created on the driver such that it can be used on the executors.
*
* All accumulators registered here can later be used as a container for accumulating partial
* values across multiple tasks. This is what `org.apache.spark.scheduler.DAGScheduler` does.
* Note: if an accumulator is registered here, it should also be registered with the active
* context cleaner for cleanup so as to avoid memory leaks.
*
* If an [[AccumulatorV2]] with the same ID was already registered, this does nothing instead
* of overwriting it. We will never register same accumulator twice, this is just a sanity check.
*/
def register(a: AccumulatorV2[_, _]): Unit = {
originals.putIfAbsent(a.id, new jl.ref.WeakReference[AccumulatorV2[_, _]](a))
}
/**
* Unregisters the [[AccumulatorV2]] with the given ID, if any.
*/
def remove(id: Long): Unit = {
originals.remove(id)
}
/**
* Returns the [[AccumulatorV2]] registered with the given ID, if any.
*/
def get(id: Long): Option[AccumulatorV2[_, _]] = {
Option(originals.get(id)).map { ref =>
// Since we are storing weak references, we must check whether the underlying data is valid.
val acc = ref.get
if (acc eq null) {
throw new IllegalStateException(s"Attempted to access garbage collected accumulator $id")
}
acc
}
}
/**
* Clears all registered [[AccumulatorV2]]s. For testing only.
*/
def clear(): Unit = {
originals.clear()
}
// Identifier for distinguishing SQL metrics from other accumulators
private[spark] val SQL_ACCUM_IDENTIFIER = "sql"
}
/**
* An [[AccumulatorV2 accumulator]] for computing sum, count, and average of 64-bit integers.
*
* @since 2.0.0
*/
class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] {
private var _sum = 0L
private var _count = 0L
/**
* Adds v to the accumulator, i.e. increment sum by v and count by 1.
* @since 2.0.0
*/
override def isZero: Boolean = _sum == 0L && _count == 0
override def copy(): LongAccumulator = {
val newAcc = new LongAccumulator
newAcc._count = this._count
newAcc._sum = this._sum
newAcc
}
override def reset(): Unit = {
_sum = 0L
_count = 0L
}
/**
* Adds v to the accumulator, i.e. increment sum by v and count by 1.
* @since 2.0.0
*/
override def add(v: jl.Long): Unit = {
_sum += v
_count += 1
}
/**
* Adds v to the accumulator, i.e. increment sum by v and count by 1.
* @since 2.0.0
*/
def add(v: Long): Unit = {
_sum += v
_count += 1
}
/**
* Returns the number of elements added to the accumulator.
* @since 2.0.0
*/
def count: Long = _count
/**
* Returns the sum of elements added to the accumulator.
* @since 2.0.0
*/
def sum: Long = _sum
/**
* Returns the average of elements added to the accumulator.
* @since 2.0.0
*/
def avg: Double = _sum.toDouble / _count
override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match {
case o: LongAccumulator =>
_sum += o.sum
_count += o.count
case _ =>
throw new UnsupportedOperationException(
s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}")
}
private[spark] def setValue(newValue: Long): Unit = _sum = newValue
override def value: jl.Long = _sum
}
/**
* An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for double precision
* floating numbers.
*
* @since 2.0.0
*/
class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] {
private var _sum = 0.0
private var _count = 0L
override def isZero: Boolean = _sum == 0.0 && _count == 0
override def copy(): DoubleAccumulator = {
val newAcc = new DoubleAccumulator
newAcc._count = this._count
newAcc._sum = this._sum
newAcc
}
override def reset(): Unit = {
_sum = 0.0
_count = 0L
}
/**
* Adds v to the accumulator, i.e. increment sum by v and count by 1.
* @since 2.0.0
*/
override def add(v: jl.Double): Unit = {
_sum += v
_count += 1
}
/**
* Adds v to the accumulator, i.e. increment sum by v and count by 1.
* @since 2.0.0
*/
def add(v: Double): Unit = {
_sum += v
_count += 1
}
/**
* Returns the number of elements added to the accumulator.
* @since 2.0.0
*/
def count: Long = _count
/**
* Returns the sum of elements added to the accumulator.
* @since 2.0.0
*/
def sum: Double = _sum
/**
* Returns the average of elements added to the accumulator.
* @since 2.0.0
*/
def avg: Double = _sum / _count
override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match {
case o: DoubleAccumulator =>
_sum += o.sum
_count += o.count
case _ =>
throw new UnsupportedOperationException(
s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}")
}
private[spark] def setValue(newValue: Double): Unit = _sum = newValue
override def value: jl.Double = _sum
}
/**
* An [[AccumulatorV2 accumulator]] for collecting a list of elements.
*
* @since 2.0.0
*/
class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] {
private val _list: java.util.List[T] = Collections.synchronizedList(new ArrayList[T]())
override def isZero: Boolean = _list.isEmpty
override def copyAndReset(): CollectionAccumulator[T] = new CollectionAccumulator
override def copy(): CollectionAccumulator[T] = {
val newAcc = new CollectionAccumulator[T]
_list.synchronized {
newAcc._list.addAll(_list)
}
newAcc
}
override def reset(): Unit = _list.clear()
override def add(v: T): Unit = _list.add(v)
override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match {
case o: CollectionAccumulator[T] => _list.addAll(o.value)
case _ => throw new UnsupportedOperationException(
s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}")
}
override def value: java.util.List[T] = _list.synchronized {
java.util.Collections.unmodifiableList(new ArrayList[T](_list))
}
private[spark] def setValue(newValue: java.util.List[T]): Unit = {
_list.clear()
_list.addAll(newValue)
}
}
class LegacyAccumulatorWrapper[R, T](
initialValue: R,
param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] {
private[spark] var _value = initialValue // Current value on driver
override def isZero: Boolean = _value == param.zero(initialValue)
override def copy(): LegacyAccumulatorWrapper[R, T] = {
val acc = new LegacyAccumulatorWrapper(initialValue, param)
acc._value = _value
acc
}
override def reset(): Unit = {
_value = param.zero(initialValue)
}
override def add(v: T): Unit = _value = param.addAccumulator(_value, v)
override def merge(other: AccumulatorV2[T, R]): Unit = other match {
case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value)
case _ => throw new UnsupportedOperationException(
s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}")
}
override def value: R = _value
}