private synchronized boolean isClusterOutOfMemory() { ClusterMemoryPool reservedPool = pools.get(RESERVED_POOL); ClusterMemoryPool generalPool = pools.get(GENERAL_POOL); if (reservedPool == null) { return generalPool.getBlockedNodes() > 0; } return reservedPool.getAssignedQueries() > 0 && generalPool.getBlockedNodes() > 0; }
private Map<MemoryPoolId, ClusterMemoryPool> createClusterMemoryPools(boolean systemPoolEnabled, boolean reservedPoolEnabled) { Set<MemoryPoolId> memoryPools = new HashSet<>(); memoryPools.add(GENERAL_POOL); if (systemPoolEnabled) { memoryPools.add(SYSTEM_POOL); } if (reservedPoolEnabled) { memoryPools.add(RESERVED_POOL); } ImmutableMap.Builder<MemoryPoolId, ClusterMemoryPool> builder = ImmutableMap.builder(); for (MemoryPoolId poolId : memoryPools) { ClusterMemoryPool pool = new ClusterMemoryPool(poolId); builder.put(poolId, pool); try { exporter.export(generatedNameOf(ClusterMemoryPool.class, poolId.toString()), pool); } catch (JmxException e) { log.error(e, "Error exporting memory pool %s", poolId); } } return builder.build(); }
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)); } } } }
ClusterMemoryPool newPool = new ClusterMemoryPool(poolId); String objectName = ObjectNames.builder(ClusterMemoryPool.class, newPool.getId().toString()).build(); try { exporter.export(objectName, newPool); pool.update(nodeMemoryInfos, queryCounts.getOrDefault(pool.getId(), 0));
@GuardedBy("this") private boolean isLastKilledQueryGone() { if (lastKilledQuery == null) { return true; } // If the lastKilledQuery is marked as leaked by the ClusterMemoryLeakDetector we consider the lastKilledQuery as gone, // so that the ClusterMemoryManager can continue to make progress even if there are leaks. // Even if the weak references to the leaked queries are GCed in the ClusterMemoryLeakDetector, it will mark the same queries // as leaked in its next run, and eventually the ClusterMemoryManager will make progress. if (memoryLeakDetector.wasQueryPossiblyLeaked(lastKilledQuery)) { 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); }
@PreDestroy public synchronized void destroy() throws IOException { try (Closer closer = Closer.create()) { for (ClusterMemoryPool pool : pools.values()) { closer.register(() -> exporter.unexport(generatedNameOf(ClusterMemoryPool.class, pool.getId().toString()))); } closer.register(listenerExecutor::shutdownNow); } }
@Override public synchronized String toString() { return toStringHelper(this) .add("id", id) .add("totalDistributedBytes", totalDistributedBytes) .add("freeDistributedBytes", getFreeDistributedBytes()) .add("reservedDistributedBytes", reservedDistributedBytes) .add("reservedRevocableDistributedBytes", reservedRevocableDistributedBytes) .add("nodes", nodes) .add("blockedNodes", blockedNodes) .add("assignedQueries", assignedQueries) .add("queryMemoryReservations", queryMemoryReservations) .add("queryMemoryAllocations", queryMemoryAllocations) .add("queryMemoryRevocableReservations", queryMemoryRevocableReservations) .toString(); } }
memoryLeakDetector.checkForMemoryLeaks(allQueryInfoSupplier, pools.get(GENERAL_POOL).getQueryMemoryReservations());
if (reservedPool.getAssignedQueries() == 0 && generalPool.getBlockedNodes() > 0) { QueryExecution biggestQuery = null; long maxMemory = -1;
ClusterMemoryPool generalPool = pools.get(GENERAL_POOL); if (generalPool != null) { lastKilledQueryIsGone = generalPool.getQueryMemoryReservations().containsKey(lastKilledQuery);
while (generalPool.getAssignedQueries() != 1 || reservedPool.getAssignedQueries() != 1 || generalPool.getBlockedNodes() != 2 || reservedPool.getBlockedNodes() != 2) { MILLISECONDS.sleep(10);
private 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; }
if (reservedPool.getAssignedQueries() == 0 && generalPool.getBlockedNodes() > 0) { QueryExecution biggestQuery = null; long maxMemory = -1;
while (generalPool.getAssignedQueries() != 1 || reservedPool.getAssignedQueries() != 1 || generalPool.getBlockedNodes() != 2 || reservedPool.getBlockedNodes() != 2) { MILLISECONDS.sleep(10);
while (generalPool.getAssignedQueries() != 1 || reservedPool.getAssignedQueries() != 1 || generalPool.getBlockedNodes() != 2 || reservedPool.getBlockedNodes() != 2) { MILLISECONDS.sleep(10);