forked from prestodb/presto
/
ClusterMemoryManager.java
568 lines (500 loc) · 24.1 KB
/
ClusterMemoryManager.java
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
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
/*
* Licensed 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 com.facebook.presto.memory;
import com.facebook.presto.execution.LocationFactory;
import com.facebook.presto.execution.QueryExecution;
import com.facebook.presto.execution.QueryIdGenerator;
import com.facebook.presto.memory.LowMemoryKiller.QueryMemoryInfo;
import com.facebook.presto.metadata.InternalNodeManager;
import com.facebook.presto.server.ServerConfig;
import com.facebook.presto.spi.Node;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.memory.ClusterMemoryPoolManager;
import com.facebook.presto.spi.memory.MemoryPoolId;
import com.facebook.presto.spi.memory.MemoryPoolInfo;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Streams;
import io.airlift.http.client.HttpClient;
import io.airlift.json.JsonCodec;
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import org.weakref.jmx.JmxException;
import org.weakref.jmx.MBeanExporter;
import org.weakref.jmx.Managed;
import org.weakref.jmx.ObjectNames;
import javax.annotation.PreDestroy;
import javax.annotation.concurrent.GuardedBy;
import javax.inject.Inject;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
import static com.facebook.presto.ExceededMemoryLimitException.exceededGlobalTotalLimit;
import static com.facebook.presto.ExceededMemoryLimitException.exceededGlobalUserLimit;
import static com.facebook.presto.SystemSessionProperties.RESOURCE_OVERCOMMIT;
import static com.facebook.presto.SystemSessionProperties.getQueryMaxMemory;
import static com.facebook.presto.SystemSessionProperties.getQueryMaxTotalMemory;
import static com.facebook.presto.SystemSessionProperties.resourceOvercommit;
import static com.facebook.presto.memory.LocalMemoryManager.GENERAL_POOL;
import static com.facebook.presto.memory.LocalMemoryManager.RESERVED_POOL;
import static com.facebook.presto.memory.LocalMemoryManager.SYSTEM_POOL;
import static com.facebook.presto.spi.NodeState.ACTIVE;
import static com.facebook.presto.spi.NodeState.SHUTTING_DOWN;
import static com.facebook.presto.spi.StandardErrorCode.CLUSTER_OUT_OF_MEMORY;
import static com.facebook.presto.spi.StandardErrorCode.EXCEEDED_MEMORY_LIMIT;
import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.MoreCollectors.toOptional;
import static com.google.common.collect.Sets.difference;
import static io.airlift.units.DataSize.succinctBytes;
import static io.airlift.units.Duration.nanosSince;
import static java.lang.Math.min;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
public class ClusterMemoryManager
implements ClusterMemoryPoolManager
{
private static final Set<MemoryPoolId> POOLS = ImmutableSet.of(GENERAL_POOL, RESERVED_POOL, SYSTEM_POOL);
private static final Logger log = Logger.get(ClusterMemoryManager.class);
private final ExecutorService listenerExecutor = Executors.newSingleThreadExecutor();
private final InternalNodeManager nodeManager;
private final LocationFactory locationFactory;
private final HttpClient httpClient;
private final MBeanExporter exporter;
private final JsonCodec<MemoryInfo> memoryInfoCodec;
private final JsonCodec<MemoryPoolAssignmentsRequest> assignmentsRequestJsonCodec;
private final DataSize maxQueryMemory;
private final DataSize maxQueryTotalMemory;
private final boolean enabled;
private final LowMemoryKiller lowMemoryKiller;
private final Duration killOnOutOfMemoryDelay;
private final String coordinatorId;
private final AtomicLong memoryPoolAssignmentsVersion = new AtomicLong();
private final AtomicLong clusterUserMemoryReservation = new AtomicLong();
private final AtomicLong clusterTotalMemoryReservation = new AtomicLong();
private final AtomicLong clusterMemoryBytes = new AtomicLong();
private final AtomicLong queriesKilledDueToOutOfMemory = new AtomicLong();
@GuardedBy("this")
public final Map<QueryId, Long> preAllocations = new HashMap<>();
@GuardedBy("this")
private final Map<QueryId, Long> preAllocationsConsumed = new HashMap<>();
@GuardedBy("this")
private final Map<String, RemoteNodeMemory> nodes = new HashMap<>();
//TODO remove when the system pool is completely removed
private final boolean isLegacySystemPoolEnabled;
@GuardedBy("this")
private final Map<MemoryPoolId, List<Consumer<MemoryPoolInfo>>> changeListeners = new HashMap<>();
@GuardedBy("this")
private final Map<MemoryPoolId, ClusterMemoryPool> pools;
@GuardedBy("this")
private long lastTimeNotOutOfMemory = System.nanoTime();
@GuardedBy("this")
private QueryId lastKilledQuery;
@Inject
public ClusterMemoryManager(
@ForMemoryManager HttpClient httpClient,
InternalNodeManager nodeManager,
LocationFactory locationFactory,
MBeanExporter exporter,
JsonCodec<MemoryInfo> memoryInfoCodec,
JsonCodec<MemoryPoolAssignmentsRequest> assignmentsRequestJsonCodec,
QueryIdGenerator queryIdGenerator,
LowMemoryKiller lowMemoryKiller,
ServerConfig serverConfig,
MemoryManagerConfig config,
NodeMemoryConfig nodeMemoryConfig)
{
requireNonNull(config, "config is null");
requireNonNull(nodeMemoryConfig, "nodeMemoryConfig is null");
requireNonNull(serverConfig, "serverConfig is null");
this.nodeManager = requireNonNull(nodeManager, "nodeManager is null");
this.locationFactory = requireNonNull(locationFactory, "locationFactory is null");
this.httpClient = requireNonNull(httpClient, "httpClient is null");
this.exporter = requireNonNull(exporter, "exporter is null");
this.memoryInfoCodec = requireNonNull(memoryInfoCodec, "memoryInfoCodec is null");
this.assignmentsRequestJsonCodec = requireNonNull(assignmentsRequestJsonCodec, "assignmentsRequestJsonCodec is null");
this.lowMemoryKiller = requireNonNull(lowMemoryKiller, "lowMemoryKiller is null");
this.maxQueryMemory = config.getMaxQueryMemory();
this.maxQueryTotalMemory = config.getMaxQueryTotalMemory();
this.coordinatorId = queryIdGenerator.getCoordinatorId();
this.enabled = serverConfig.isCoordinator();
this.killOnOutOfMemoryDelay = config.getKillOnOutOfMemoryDelay();
this.isLegacySystemPoolEnabled = nodeMemoryConfig.isLegacySystemPoolEnabled();
verify(maxQueryMemory.toBytes() <= maxQueryTotalMemory.toBytes(),
"maxQueryMemory cannot be greater than maxQueryTotalMemory");
ImmutableMap.Builder<MemoryPoolId, ClusterMemoryPool> builder = ImmutableMap.builder();
for (MemoryPoolId poolId : POOLS) {
ClusterMemoryPool pool = new ClusterMemoryPool(poolId);
builder.put(poolId, pool);
String objectName = ObjectNames.builder(ClusterMemoryPool.class, poolId.toString()).build();
try {
exporter.export(objectName, pool);
}
catch (JmxException e) {
log.error(e, "Error exporting memory pool %s", poolId);
}
}
this.pools = builder.build();
}
@Override
public synchronized void addChangeListener(MemoryPoolId poolId, Consumer<MemoryPoolInfo> listener)
{
changeListeners.computeIfAbsent(poolId, id -> new ArrayList<>()).add(listener);
}
public synchronized void process(Iterable<QueryExecution> queries)
{
if (!enabled) {
return;
}
boolean outOfMemory = isClusterOutOfMemory();
if (!outOfMemory) {
lastTimeNotOutOfMemory = System.nanoTime();
}
preAllocationsConsumed.clear();
boolean queryKilled = false;
long totalUserMemoryBytes = 0L;
long totalMemoryBytes = 0L;
for (QueryExecution query : queries) {
boolean resourceOvercommit = resourceOvercommit(query.getSession());
long userMemoryReservation = query.getUserMemoryReservation();
long totalMemoryReservation = query.getTotalMemoryReservation();
if (resourceOvercommit && outOfMemory) {
// If a query has requested resource overcommit, only kill it if the cluster has run out of memory
DataSize memory = succinctBytes(getQueryMemoryReservation(query));
query.fail(new PrestoException(CLUSTER_OUT_OF_MEMORY,
format("The cluster is out of memory and %s=true, so this query was killed. It was using %s of memory", RESOURCE_OVERCOMMIT, memory)));
queryKilled = true;
}
if (!resourceOvercommit) {
long userMemoryLimit = min(maxQueryMemory.toBytes(), getQueryMaxMemory(query.getSession()).toBytes());
if (userMemoryReservation > userMemoryLimit) {
query.fail(exceededGlobalUserLimit(succinctBytes(userMemoryLimit)));
queryKilled = true;
}
// enforce global total memory limit if system pool is disabled
long totalMemoryLimit = min(maxQueryTotalMemory.toBytes(), getQueryMaxTotalMemory(query.getSession()).toBytes());
if (!isLegacySystemPoolEnabled && totalMemoryReservation > totalMemoryLimit) {
query.fail(exceededGlobalTotalLimit(succinctBytes(totalMemoryLimit)));
queryKilled = true;
}
}
if (preAllocations.containsKey(query.getQueryId())) {
preAllocationsConsumed.put(query.getQueryId(), userMemoryReservation);
}
totalUserMemoryBytes += userMemoryReservation;
totalMemoryBytes += totalMemoryReservation;
}
clusterUserMemoryReservation.set(totalUserMemoryBytes);
clusterTotalMemoryReservation.set(totalMemoryBytes);
if (!(lowMemoryKiller instanceof NoneLowMemoryKiller) &&
outOfMemory &&
!queryKilled &&
nanosSince(lastTimeNotOutOfMemory).compareTo(killOnOutOfMemoryDelay) > 0) {
boolean lastKilledQueryGone = isLastKilledQueryGone();
if (lastKilledQueryGone) {
callOomKiller(queries);
}
else {
log.debug("Last killed query is still not gone: %s", lastKilledQuery);
}
}
Map<MemoryPoolId, Integer> countByPool = new HashMap<>();
for (QueryExecution query : queries) {
MemoryPoolId id = query.getMemoryPool().getId();
countByPool.put(id, countByPool.getOrDefault(id, 0) + 1);
}
updatePools(countByPool);
updateNodes(updateAssignments(queries));
}
private synchronized void callOomKiller(Iterable<QueryExecution> queries)
{
List<QueryMemoryInfo> queryMemoryInfoList = Streams.stream(queries)
.map(this::createQueryMemoryInfo)
.collect(toImmutableList());
List<MemoryInfo> nodeMemoryInfos = nodes.values().stream()
.map(RemoteNodeMemory::getInfo)
.filter(Optional::isPresent)
.map(Optional::get)
.collect(toImmutableList());
Optional<QueryId> chosenQueryId = lowMemoryKiller.chooseQueryToKill(queryMemoryInfoList, nodeMemoryInfos);
if (chosenQueryId.isPresent()) {
log.debug("Low memory killer chose %s", chosenQueryId.get());
Optional<QueryExecution> chosenQuery = Streams.stream(queries).filter(query -> chosenQueryId.get().equals(query.getQueryId())).collect(toOptional());
if (chosenQuery.isPresent()) {
// See comments in isLastKilledQueryGone for why chosenQuery might be absent.
chosenQuery.get().fail(new PrestoException(CLUSTER_OUT_OF_MEMORY, "Query killed because the cluster is out of memory. Please try again in a few minutes."));
queriesKilledDueToOutOfMemory.incrementAndGet();
lastKilledQuery = chosenQueryId.get();
logQueryKill(chosenQueryId.get(), nodeMemoryInfos);
}
}
}
@GuardedBy("this")
private boolean isLastKilledQueryGone()
{
if (lastKilledQuery == null) {
return true;
}
// pools fields is updated based on nodes field.
// Therefore, if the query is gone from pools field, it should also be gone from nodes field.
// However, since nodes can updated asynchronously, it has the potential of coming back after being gone.
// Therefore, even if the query appears to be gone here, it might be back when one inspects nodes later.
return !pools.get(GENERAL_POOL)
.getQueryMemoryReservations()
.containsKey(lastKilledQuery);
}
private void logQueryKill(QueryId killedQueryId, List<MemoryInfo> nodes)
{
if (!log.isInfoEnabled()) {
return;
}
StringBuilder nodeDescription = new StringBuilder();
nodeDescription.append("Query Kill Decision: Killed ").append(killedQueryId).append("\n");
for (MemoryInfo node : nodes) {
MemoryPoolInfo memoryPoolInfo = node.getPools().get(GENERAL_POOL);
if (memoryPoolInfo == null) {
continue;
}
nodeDescription.append("Query Kill Scenario: ");
nodeDescription.append("MaxBytes ").append(memoryPoolInfo.getMaxBytes()).append(' ');
nodeDescription.append("FreeBytes ").append(memoryPoolInfo.getFreeBytes() + memoryPoolInfo.getReservedRevocableBytes()).append(' ');
nodeDescription.append("Queries ");
Joiner.on(",").withKeyValueSeparator("=").appendTo(nodeDescription, memoryPoolInfo.getQueryMemoryReservations());
nodeDescription.append('\n');
}
log.info(nodeDescription.toString());
}
public synchronized boolean preAllocateQueryMemory(QueryId queryId, long requiredBytes)
{
if (requiredBytes > maxQueryMemory.toBytes()) {
throw new PrestoException(EXCEEDED_MEMORY_LIMIT, format("Cannot pre-allocate memory, exceeds maximum limit %s", maxQueryMemory));
}
ClusterMemoryPool generalPool = pools.get(GENERAL_POOL);
ClusterMemoryPool reservedPool = pools.get(RESERVED_POOL);
if (generalPool.getBlockedNodes() > 0 || reservedPool.getAssignedQueries() > 0) {
return false;
}
long totalPreAllocation = preAllocations.values().stream()
.mapToLong(Long::longValue)
.sum();
long totalPreAllocationConsumed = preAllocationsConsumed.values().stream()
.mapToLong(Long::longValue)
.sum();
if (generalPool.getFreeDistributedBytes() - (totalPreAllocation - totalPreAllocationConsumed) >= requiredBytes) {
preAllocations.put(queryId, requiredBytes);
return true;
}
return false;
}
public synchronized void removePreAllocation(QueryId queryId)
{
preAllocations.remove(queryId);
MemoryPoolInfo info = pools.get(GENERAL_POOL).getInfo();
for (Consumer<MemoryPoolInfo> listener : changeListeners.get(GENERAL_POOL)) {
listenerExecutor.execute(() -> listener.accept(info));
}
}
@VisibleForTesting
synchronized Map<MemoryPoolId, ClusterMemoryPool> getPools()
{
return ImmutableMap.copyOf(pools);
}
private synchronized boolean isClusterOutOfMemory()
{
ClusterMemoryPool reservedPool = pools.get(RESERVED_POOL);
ClusterMemoryPool generalPool = pools.get(GENERAL_POOL);
return reservedPool != null && generalPool != null && reservedPool.getAssignedQueries() > 0 && generalPool.getBlockedNodes() > 0;
}
private synchronized MemoryPoolAssignmentsRequest updateAssignments(Iterable<QueryExecution> queries)
{
ClusterMemoryPool reservedPool = pools.get(RESERVED_POOL);
ClusterMemoryPool generalPool = pools.get(GENERAL_POOL);
long version = memoryPoolAssignmentsVersion.incrementAndGet();
// Check that all previous assignments have propagated to the visible nodes. This doesn't account for temporary network issues,
// and is more of a safety check than a guarantee
if (reservedPool != null && generalPool != null && allAssignmentsHavePropagated(queries)) {
if (reservedPool.getAssignedQueries() == 0 && generalPool.getBlockedNodes() > 0) {
QueryExecution biggestQuery = null;
long maxMemory = -1;
for (QueryExecution queryExecution : queries) {
if (resourceOvercommit(queryExecution.getSession())) {
// Don't promote queries that requested resource overcommit to the reserved pool,
// since their memory usage is unbounded.
continue;
}
long bytesUsed = getQueryMemoryReservation(queryExecution);
if (bytesUsed > maxMemory) {
biggestQuery = queryExecution;
maxMemory = bytesUsed;
}
}
if (biggestQuery != null) {
biggestQuery.setMemoryPool(new VersionedMemoryPoolId(RESERVED_POOL, version));
}
}
}
ImmutableList.Builder<MemoryPoolAssignment> assignments = ImmutableList.builder();
for (QueryExecution queryExecution : queries) {
assignments.add(new MemoryPoolAssignment(queryExecution.getQueryId(), queryExecution.getMemoryPool().getId()));
}
return new MemoryPoolAssignmentsRequest(coordinatorId, version, assignments.build());
}
private QueryMemoryInfo createQueryMemoryInfo(QueryExecution query)
{
// when the legacy system pool is enabled we use the user memory instead of the total memory
if (isLegacySystemPoolEnabled) {
return new QueryMemoryInfo(query.getQueryId(), query.getMemoryPool().getId(), query.getUserMemoryReservation());
}
return new QueryMemoryInfo(query.getQueryId(), query.getMemoryPool().getId(), query.getTotalMemoryReservation());
}
private long getQueryMemoryReservation(QueryExecution query)
{
// when the legacy system pool is enabled we use the user memory instead of the total memory
if (isLegacySystemPoolEnabled) {
return query.getUserMemoryReservation();
}
return query.getTotalMemoryReservation();
}
private synchronized boolean allAssignmentsHavePropagated(Iterable<QueryExecution> queries)
{
if (nodes.isEmpty()) {
// Assignments can't have propagated, if there are no visible nodes.
return false;
}
long newestAssignment = ImmutableList.copyOf(queries).stream()
.map(QueryExecution::getMemoryPool)
.mapToLong(VersionedMemoryPoolId::getVersion)
.min()
.orElse(-1);
long mostOutOfDateNode = nodes.values().stream()
.mapToLong(RemoteNodeMemory::getCurrentAssignmentVersion)
.min()
.orElse(Long.MAX_VALUE);
return newestAssignment <= mostOutOfDateNode;
}
private synchronized void updateNodes(MemoryPoolAssignmentsRequest assignments)
{
ImmutableSet.Builder<Node> builder = ImmutableSet.builder();
Set<Node> aliveNodes = builder
.addAll(nodeManager.getNodes(ACTIVE))
.addAll(nodeManager.getNodes(SHUTTING_DOWN))
.build();
ImmutableSet<String> aliveNodeIds = aliveNodes.stream()
.map(Node::getNodeIdentifier)
.collect(toImmutableSet());
// Remove nodes that don't exist anymore
// Make a copy to materialize the set difference
Set<String> deadNodes = ImmutableSet.copyOf(difference(nodes.keySet(), aliveNodeIds));
nodes.keySet().removeAll(deadNodes);
// Add new nodes
for (Node node : aliveNodes) {
if (!nodes.containsKey(node.getNodeIdentifier())) {
nodes.put(node.getNodeIdentifier(), new RemoteNodeMemory(node, httpClient, memoryInfoCodec, assignmentsRequestJsonCodec, locationFactory.createMemoryInfoLocation(node)));
}
}
// Schedule refresh
for (RemoteNodeMemory node : nodes.values()) {
node.asyncRefresh(assignments);
}
}
private synchronized void updatePools(Map<MemoryPoolId, Integer> queryCounts)
{
// Update view of cluster memory and pools
List<MemoryInfo> nodeMemoryInfos = nodes.values().stream()
.map(RemoteNodeMemory::getInfo)
.filter(Optional::isPresent)
.map(Optional::get)
.collect(toImmutableList());
long totalClusterMemory = nodeMemoryInfos.stream()
.map(MemoryInfo::getTotalNodeMemory)
.mapToLong(DataSize::toBytes)
.sum();
clusterMemoryBytes.set(totalClusterMemory);
for (ClusterMemoryPool pool : pools.values()) {
pool.update(nodeMemoryInfos, queryCounts.getOrDefault(pool.getId(), 0));
if (changeListeners.containsKey(pool.getId())) {
MemoryPoolInfo info = pool.getInfo();
for (Consumer<MemoryPoolInfo> listener : changeListeners.get(pool.getId())) {
listenerExecutor.execute(() -> listener.accept(info));
}
}
}
}
public synchronized Map<String, Optional<MemoryInfo>> getWorkerMemoryInfo()
{
Map<String, Optional<MemoryInfo>> memoryInfo = new HashMap<>();
for (Entry<String, RemoteNodeMemory> entry : nodes.entrySet()) {
// workerId is of the form "node_identifier [node_host]"
String workerId = entry.getKey() + " [" + entry.getValue().getNode().getHostAndPort().getHostText() + "]";
memoryInfo.put(workerId, entry.getValue().getInfo());
}
return memoryInfo;
}
@PreDestroy
public synchronized void destroy()
{
try {
for (ClusterMemoryPool pool : pools.values()) {
unexport(pool);
}
}
finally {
listenerExecutor.shutdownNow();
}
}
private void unexport(ClusterMemoryPool pool)
{
try {
String objectName = ObjectNames.builder(ClusterMemoryPool.class, pool.getId().toString()).build();
exporter.unexport(objectName);
}
catch (JmxException e) {
log.error(e, "Failed to unexport pool %s", pool.getId());
}
}
@Managed
public long getClusterUserMemoryReservation()
{
return clusterUserMemoryReservation.get();
}
@Managed
public long getClusterTotalMemoryReservation()
{
return clusterTotalMemoryReservation.get();
}
@Managed
public long getClusterMemoryBytes()
{
return clusterMemoryBytes.get();
}
@Managed
public long getQueriesKilledDueToOutOfMemory()
{
return queriesKilledDueToOutOfMemory.get();
}
}