-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
KafkaRequestHandler.scala
executable file
·482 lines (402 loc) · 21.3 KB
/
KafkaRequestHandler.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
/**
* 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 kafka.server
import kafka.network._
import kafka.utils._
import kafka.server.KafkaRequestHandler.{threadCurrentRequest, threadRequestChannel}
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.concurrent.atomic.AtomicInteger
import com.yammer.metrics.core.Meter
import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.utils.{KafkaThread, Time}
import org.apache.kafka.server.log.remote.storage.RemoteStorageMetrics
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import java.util.Collections
import scala.collection.mutable
import scala.jdk.CollectionConverters._
trait ApiRequestHandler {
def handle(request: RequestChannel.Request, requestLocal: RequestLocal): Unit
}
object KafkaRequestHandler {
// Support for scheduling callbacks on a request thread.
private val threadRequestChannel = new ThreadLocal[RequestChannel]
private val threadCurrentRequest = new ThreadLocal[RequestChannel.Request]
// For testing
@volatile private var bypassThreadCheck = false
def setBypassThreadCheck(bypassCheck: Boolean): Unit = {
bypassThreadCheck = bypassCheck
}
def currentRequestOnThread(): RequestChannel.Request = {
threadCurrentRequest.get()
}
/**
* Wrap callback to schedule it on a request thread.
* NOTE: this function must be called on a request thread.
* @param fun Callback function to execute
* @return Wrapped callback that would execute `fun` on a request thread
*/
def wrap[T](fun: T => Unit): T => Unit = {
val requestChannel = threadRequestChannel.get()
val currentRequest = threadCurrentRequest.get()
if (requestChannel == null || currentRequest == null) {
if (!bypassThreadCheck)
throw new IllegalStateException("Attempted to reschedule to request handler thread from non-request handler thread.")
T => fun(T)
} else {
T => {
// The requestChannel and request are captured in this lambda, so when it's executed on the callback thread
// we can re-schedule the original callback on a request thread and update the metrics accordingly.
requestChannel.sendCallbackRequest(RequestChannel.CallbackRequest(() => fun(T), currentRequest))
}
}
}
}
/**
* A thread that answers kafka requests.
*/
class KafkaRequestHandler(id: Int,
brokerId: Int,
val aggregateIdleMeter: Meter,
val totalHandlerThreads: AtomicInteger,
val requestChannel: RequestChannel,
apis: ApiRequestHandler,
time: Time) extends Runnable with Logging {
this.logIdent = s"[Kafka Request Handler $id on Broker $brokerId], "
private val shutdownComplete = new CountDownLatch(1)
private val requestLocal = RequestLocal.withThreadConfinedCaching
@volatile private var stopped = false
def run(): Unit = {
threadRequestChannel.set(requestChannel)
while (!stopped) {
// We use a single meter for aggregate idle percentage for the thread pool.
// Since meter is calculated as total_recorded_value / time_window and
// time_window is independent of the number of threads, each recorded idle
// time should be discounted by # threads.
val startSelectTime = time.nanoseconds
val req = requestChannel.receiveRequest(300)
val endTime = time.nanoseconds
val idleTime = endTime - startSelectTime
aggregateIdleMeter.mark(idleTime / totalHandlerThreads.get)
req match {
case RequestChannel.ShutdownRequest =>
debug(s"Kafka request handler $id on broker $brokerId received shut down command")
completeShutdown()
return
case callback: RequestChannel.CallbackRequest =>
try {
val originalRequest = callback.originalRequest
// If we've already executed a callback for this request, reset the times and subtract the callback time from the
// new dequeue time. This will allow calculation of multiple callback times.
// Otherwise, set dequeue time to now.
if (originalRequest.callbackRequestDequeueTimeNanos.isDefined) {
val prevCallbacksTimeNanos = originalRequest.callbackRequestCompleteTimeNanos.getOrElse(0L) - originalRequest.callbackRequestDequeueTimeNanos.getOrElse(0L)
originalRequest.callbackRequestCompleteTimeNanos = None
originalRequest.callbackRequestDequeueTimeNanos = Some(time.nanoseconds() - prevCallbacksTimeNanos)
} else {
originalRequest.callbackRequestDequeueTimeNanos = Some(time.nanoseconds())
}
threadCurrentRequest.set(originalRequest)
callback.fun()
if (originalRequest.callbackRequestCompleteTimeNanos.isEmpty)
originalRequest.callbackRequestCompleteTimeNanos = Some(time.nanoseconds())
} catch {
case e: FatalExitError =>
completeShutdown()
Exit.exit(e.statusCode)
case e: Throwable => error("Exception when handling request", e)
} finally {
threadCurrentRequest.remove()
}
case request: RequestChannel.Request =>
try {
request.requestDequeueTimeNanos = endTime
trace(s"Kafka request handler $id on broker $brokerId handling request $request")
threadCurrentRequest.set(request)
apis.handle(request, requestLocal)
} catch {
case e: FatalExitError =>
completeShutdown()
Exit.exit(e.statusCode)
case e: Throwable => error("Exception when handling request", e)
} finally {
threadCurrentRequest.remove()
request.releaseBuffer()
}
case RequestChannel.WakeupRequest =>
// We should handle this in receiveRequest by polling callbackQueue.
warn("Received a wakeup request outside of typical usage.")
case null => // continue
}
}
completeShutdown()
}
private def completeShutdown(): Unit = {
requestLocal.close()
shutdownComplete.countDown()
}
def stop(): Unit = {
stopped = true
}
def initiateShutdown(): Unit = requestChannel.sendShutdownRequest()
def awaitShutdown(): Unit = shutdownComplete.await()
}
class KafkaRequestHandlerPool(val brokerId: Int,
val requestChannel: RequestChannel,
val apis: ApiRequestHandler,
time: Time,
numThreads: Int,
requestHandlerAvgIdleMetricName: String,
logAndThreadNamePrefix : String) extends Logging {
private val metricsGroup = new KafkaMetricsGroup(this.getClass)
private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
/* a meter to track the average free capacity of the request handlers */
private val aggregateIdleMeter = metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", TimeUnit.NANOSECONDS)
this.logIdent = "[" + logAndThreadNamePrefix + " Kafka Request Handler on Broker " + brokerId + "], "
val runnables = new mutable.ArrayBuffer[KafkaRequestHandler](numThreads)
for (i <- 0 until numThreads) {
createHandler(i)
}
def createHandler(id: Int): Unit = synchronized {
runnables += new KafkaRequestHandler(id, brokerId, aggregateIdleMeter, threadPoolSize, requestChannel, apis, time)
KafkaThread.daemon(logAndThreadNamePrefix + "-kafka-request-handler-" + id, runnables(id)).start()
}
def resizeThreadPool(newSize: Int): Unit = synchronized {
val currentSize = threadPoolSize.get
info(s"Resizing request handler thread pool size from $currentSize to $newSize")
if (newSize > currentSize) {
for (i <- currentSize until newSize) {
createHandler(i)
}
} else if (newSize < currentSize) {
for (i <- 1 to (currentSize - newSize)) {
runnables.remove(currentSize - i).stop()
}
}
threadPoolSize.set(newSize)
}
def shutdown(): Unit = synchronized {
info("shutting down")
for (handler <- runnables)
handler.initiateShutdown()
for (handler <- runnables)
handler.awaitShutdown()
info("shut down completely")
}
}
class BrokerTopicMetrics(name: Option[String], configOpt: java.util.Optional[KafkaConfig]) {
private val metricsGroup = new KafkaMetricsGroup(this.getClass)
val tags: java.util.Map[String, String] = name match {
case None => Collections.emptyMap()
case Some(topic) => Map("topic" -> topic).asJava
}
case class MeterWrapper(metricType: String, eventType: String) {
@volatile private var lazyMeter: Meter = _
private val meterLock = new Object
def meter(): Meter = {
var meter = lazyMeter
if (meter == null) {
meterLock synchronized {
meter = lazyMeter
if (meter == null) {
meter = metricsGroup.newMeter(metricType, eventType, TimeUnit.SECONDS, tags)
lazyMeter = meter
}
}
}
meter
}
def close(): Unit = meterLock synchronized {
if (lazyMeter != null) {
metricsGroup.removeMetric(metricType, tags)
lazyMeter = null
}
}
if (tags.isEmpty) // greedily initialize the general topic metrics
meter()
}
// an internal map for "lazy initialization" of certain metrics
private val metricTypeMap = new Pool[String, MeterWrapper]()
metricTypeMap.putAll(Map(
BrokerTopicStats.MessagesInPerSec -> MeterWrapper(BrokerTopicStats.MessagesInPerSec, "messages"),
BrokerTopicStats.BytesInPerSec -> MeterWrapper(BrokerTopicStats.BytesInPerSec, "bytes"),
BrokerTopicStats.BytesOutPerSec -> MeterWrapper(BrokerTopicStats.BytesOutPerSec, "bytes"),
BrokerTopicStats.BytesRejectedPerSec -> MeterWrapper(BrokerTopicStats.BytesRejectedPerSec, "bytes"),
BrokerTopicStats.FailedProduceRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedProduceRequestsPerSec, "requests"),
BrokerTopicStats.FailedFetchRequestsPerSec -> MeterWrapper(BrokerTopicStats.FailedFetchRequestsPerSec, "requests"),
BrokerTopicStats.TotalProduceRequestsPerSec -> MeterWrapper(BrokerTopicStats.TotalProduceRequestsPerSec, "requests"),
BrokerTopicStats.TotalFetchRequestsPerSec -> MeterWrapper(BrokerTopicStats.TotalFetchRequestsPerSec, "requests"),
BrokerTopicStats.FetchMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.FetchMessageConversionsPerSec, "requests"),
BrokerTopicStats.ProduceMessageConversionsPerSec -> MeterWrapper(BrokerTopicStats.ProduceMessageConversionsPerSec, "requests"),
BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec -> MeterWrapper(BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec, "requests"),
BrokerTopicStats.InvalidMagicNumberRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMagicNumberRecordsPerSec, "requests"),
BrokerTopicStats.InvalidMessageCrcRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidMessageCrcRecordsPerSec, "requests"),
BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec -> MeterWrapper(BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec, "requests")
).asJava)
if (name.isEmpty) {
metricTypeMap.put(BrokerTopicStats.ReplicationBytesInPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesInPerSec, "bytes"))
metricTypeMap.put(BrokerTopicStats.ReplicationBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReplicationBytesOutPerSec, "bytes"))
metricTypeMap.put(BrokerTopicStats.ReassignmentBytesInPerSec, MeterWrapper(BrokerTopicStats.ReassignmentBytesInPerSec, "bytes"))
metricTypeMap.put(BrokerTopicStats.ReassignmentBytesOutPerSec, MeterWrapper(BrokerTopicStats.ReassignmentBytesOutPerSec, "bytes"))
}
configOpt.ifPresent(config =>
if (config.remoteLogManagerConfig.enableRemoteStorageSystem()) {
metricTypeMap.putAll(Map(
RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName, "bytes"),
RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName, "bytes"),
RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName, "requests"),
RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName, "requests"),
RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName, "requests"),
RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName -> MeterWrapper(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName, "requests")
).asJava)
})
// used for testing only
def metricMap: Map[String, MeterWrapper] = metricTypeMap.toMap
def messagesInRate: Meter = metricTypeMap.get(BrokerTopicStats.MessagesInPerSec).meter()
def bytesInRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesInPerSec).meter()
def bytesOutRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesOutPerSec).meter()
def bytesRejectedRate: Meter = metricTypeMap.get(BrokerTopicStats.BytesRejectedPerSec).meter()
private[server] def replicationBytesInRate: Option[Meter] =
if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReplicationBytesInPerSec).meter())
else None
private[server] def replicationBytesOutRate: Option[Meter] =
if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReplicationBytesOutPerSec).meter())
else None
private[server] def reassignmentBytesInPerSec: Option[Meter] =
if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReassignmentBytesInPerSec).meter())
else None
private[server] def reassignmentBytesOutPerSec: Option[Meter] =
if (name.isEmpty) Some(metricTypeMap.get(BrokerTopicStats.ReassignmentBytesOutPerSec).meter())
else None
def failedProduceRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.FailedProduceRequestsPerSec).meter()
def failedFetchRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.FailedFetchRequestsPerSec).meter()
def totalProduceRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.TotalProduceRequestsPerSec).meter()
def totalFetchRequestRate: Meter = metricTypeMap.get(BrokerTopicStats.TotalFetchRequestsPerSec).meter()
def fetchMessageConversionsRate: Meter = metricTypeMap.get(BrokerTopicStats.FetchMessageConversionsPerSec).meter()
def produceMessageConversionsRate: Meter = metricTypeMap.get(BrokerTopicStats.ProduceMessageConversionsPerSec).meter()
def noKeyCompactedTopicRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.NoKeyCompactedTopicRecordsPerSec).meter()
def invalidMagicNumberRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidMagicNumberRecordsPerSec).meter()
def invalidMessageCrcRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidMessageCrcRecordsPerSec).meter()
def invalidOffsetOrSequenceRecordsPerSec: Meter = metricTypeMap.get(BrokerTopicStats.InvalidOffsetOrSequenceRecordsPerSec).meter()
def remoteCopyBytesRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName).meter()
def remoteFetchBytesRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName).meter()
def remoteFetchRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName).meter()
def remoteCopyRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName).meter()
def failedRemoteFetchRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName).meter()
def failedRemoteCopyRequestRate: Meter = metricTypeMap.get(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName).meter()
def closeMetric(metricType: String): Unit = {
val meter = metricTypeMap.get(metricType)
if (meter != null)
meter.close()
}
def close(): Unit = metricTypeMap.values.foreach(_.close())
}
object BrokerTopicStats {
val MessagesInPerSec = "MessagesInPerSec"
val BytesInPerSec = "BytesInPerSec"
val BytesOutPerSec = "BytesOutPerSec"
val BytesRejectedPerSec = "BytesRejectedPerSec"
val ReplicationBytesInPerSec = "ReplicationBytesInPerSec"
val ReplicationBytesOutPerSec = "ReplicationBytesOutPerSec"
val FailedProduceRequestsPerSec = "FailedProduceRequestsPerSec"
val FailedFetchRequestsPerSec = "FailedFetchRequestsPerSec"
val TotalProduceRequestsPerSec = "TotalProduceRequestsPerSec"
val TotalFetchRequestsPerSec = "TotalFetchRequestsPerSec"
val FetchMessageConversionsPerSec = "FetchMessageConversionsPerSec"
val ProduceMessageConversionsPerSec = "ProduceMessageConversionsPerSec"
val ReassignmentBytesInPerSec = "ReassignmentBytesInPerSec"
val ReassignmentBytesOutPerSec = "ReassignmentBytesOutPerSec"
// These following topics are for LogValidator for better debugging on failed records
val NoKeyCompactedTopicRecordsPerSec = "NoKeyCompactedTopicRecordsPerSec"
val InvalidMagicNumberRecordsPerSec = "InvalidMagicNumberRecordsPerSec"
val InvalidMessageCrcRecordsPerSec = "InvalidMessageCrcRecordsPerSec"
val InvalidOffsetOrSequenceRecordsPerSec = "InvalidOffsetOrSequenceRecordsPerSec"
}
class BrokerTopicStats(configOpt: java.util.Optional[KafkaConfig] = java.util.Optional.empty()) extends Logging {
private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k), configOpt)
private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory))
val allTopicsStats = new BrokerTopicMetrics(None, configOpt)
def topicStats(topic: String): BrokerTopicMetrics =
stats.getAndMaybePut(topic)
def updateReplicationBytesIn(value: Long): Unit = {
allTopicsStats.replicationBytesInRate.foreach { metric =>
metric.mark(value)
}
}
private def updateReplicationBytesOut(value: Long): Unit = {
allTopicsStats.replicationBytesOutRate.foreach { metric =>
metric.mark(value)
}
}
def updateReassignmentBytesIn(value: Long): Unit = {
allTopicsStats.reassignmentBytesInPerSec.foreach { metric =>
metric.mark(value)
}
}
def updateReassignmentBytesOut(value: Long): Unit = {
allTopicsStats.reassignmentBytesOutPerSec.foreach { metric =>
metric.mark(value)
}
}
// This method only removes metrics only used for leader
def removeOldLeaderMetrics(topic: String): Unit = {
val topicMetrics = topicStats(topic)
if (topicMetrics != null) {
topicMetrics.closeMetric(BrokerTopicStats.MessagesInPerSec)
topicMetrics.closeMetric(BrokerTopicStats.BytesInPerSec)
topicMetrics.closeMetric(BrokerTopicStats.BytesRejectedPerSec)
topicMetrics.closeMetric(BrokerTopicStats.FailedProduceRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.TotalProduceRequestsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ProduceMessageConversionsPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ReplicationBytesOutPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ReassignmentBytesOutPerSec)
topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_COPY_BYTES_PER_SEC_METRIC.getName)
topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_FETCH_BYTES_PER_SEC_METRIC.getName)
topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_FETCH_REQUESTS_PER_SEC_METRIC.getName)
topicMetrics.closeMetric(RemoteStorageMetrics.REMOTE_COPY_REQUESTS_PER_SEC_METRIC.getName)
topicMetrics.closeMetric(RemoteStorageMetrics.FAILED_REMOTE_FETCH_PER_SEC_METRIC.getName)
topicMetrics.closeMetric(RemoteStorageMetrics.FAILED_REMOTE_COPY_PER_SEC_METRIC.getName)
}
}
// This method only removes metrics only used for follower
def removeOldFollowerMetrics(topic: String): Unit = {
val topicMetrics = topicStats(topic)
if (topicMetrics != null) {
topicMetrics.closeMetric(BrokerTopicStats.ReplicationBytesInPerSec)
topicMetrics.closeMetric(BrokerTopicStats.ReassignmentBytesInPerSec)
}
}
def removeMetrics(topic: String): Unit = {
val metrics = stats.remove(topic)
if (metrics != null)
metrics.close()
}
def updateBytesOut(topic: String, isFollower: Boolean, isReassignment: Boolean, value: Long): Unit = {
if (isFollower) {
if (isReassignment)
updateReassignmentBytesOut(value)
updateReplicationBytesOut(value)
} else {
topicStats(topic).bytesOutRate.mark(value)
allTopicsStats.bytesOutRate.mark(value)
}
}
def close(): Unit = {
allTopicsStats.close()
stats.values.foreach(_.close())
info("Broker and topic stats closed")
}
}