synchronized ListenableFuture<?> moveQuery(QueryId queryId, MemoryPool targetMemoryPool) { long originalReserved = getQueryMemoryReservation(queryId); long originalRevocableReserved = getQueryRevocableMemoryReservation(queryId); // Get the tags before we call free() as that would remove the tags and we will lose the tags. Map<String, Long> taggedAllocations = taggedMemoryAllocations.remove(queryId); ListenableFuture<?> future = targetMemoryPool.reserve(queryId, MOVE_QUERY_TAG, originalReserved); free(queryId, MOVE_QUERY_TAG, originalReserved); targetMemoryPool.reserveRevocable(queryId, originalRevocableReserved); freeRevocable(queryId, originalRevocableReserved); targetMemoryPool.taggedMemoryAllocations.put(queryId, taggedAllocations); return future; }
private synchronized ListenableFuture<?> updateSystemMemory(String allocationTag, long delta) { // We call memoryPool.getQueryMemoryReservation(queryId) instead of calling queryMemoryContext.getUserMemory() to // calculate the total memory size. // // Calling the latter can result in a deadlock: // * A thread doing a user allocation will acquire locks in this order: // 1. monitor of queryMemoryContext.userAggregateMemoryContext // 2. monitor of this (QueryContext) // * The current thread doing a system allocation will acquire locks in this order: // 1. monitor of this (QueryContext) // 2. monitor of queryMemoryContext.userAggregateMemoryContext // Deadlock is possible for concurrent user and system allocations when updateSystemMemory()/updateUserMemory // calls queryMemoryContext.getUserMemory()/queryMemoryContext.getSystemMemory(), respectively. For concurrent // allocations of the same type (e.g., tryUpdateUserMemory/updateUserMemory) it is not possible as they share // the same RootAggregatedMemoryContext instance, and one of the threads will be blocked on the monitor of that // RootAggregatedMemoryContext instance even before calling the QueryContext methods (the monitors of // RootAggregatedMemoryContext instance and this will be acquired in the same order). long totalMemory = memoryPool.getQueryMemoryReservation(queryId); if (delta >= 0) { enforceTotalMemoryLimit(totalMemory, delta, maxTotalMemory); return memoryPool.reserve(queryId, allocationTag, delta); } memoryPool.free(queryId, allocationTag, -delta); return NOT_BLOCKED; }