private synchronized ListenableFuture<?> updateSystemMemory(String allocationTag, long delta) { if (delta >= 0) { systemMemoryPool.reserve(queryId, allocationTag, delta); // Since various operators and the output buffers now support blocking when the system pool is full // we return NOT_BLOCKED to prevent them from blocking, which is the legacy behavior. return NOT_BLOCKED; } systemMemoryPool.free(queryId, allocationTag, -delta); return NOT_BLOCKED; }
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<?> updateUserMemory(String allocationTag, long delta) { if (delta >= 0) { enforceUserMemoryLimit(queryMemoryContext.getUserMemory(), delta, maxMemory); return memoryPool.reserve(queryId, allocationTag, delta); } memoryPool.free(queryId, allocationTag, -delta); return NOT_BLOCKED; }
/** * Deadlock is possible for concurrent user and system allocations when updateSystemMemory()/updateUserMemory * calls queryMemoryContext.getUserMemory()/queryMemoryContext.getSystemMemory(), respectively. * * @see this##updateSystemMemory(long) for details. */ private synchronized ListenableFuture<?> updateUserMemory(String allocationTag, long delta) { if (delta >= 0) { enforceUserMemoryLimit(queryMemoryContext.getUserMemory(), delta, maxUserMemory); return memoryPool.reserve(queryId, allocationTag, delta); } memoryPool.free(queryId, allocationTag, -delta); return NOT_BLOCKED; }
@Test public void testNotifyListenerOnMemoryReserved() { setupConsumeRevocableMemory(ONE_BYTE, 10); AtomicReference<MemoryPool> notifiedPool = new AtomicReference<>(); AtomicLong notifiedBytes = new AtomicLong(); userPool.addListener(MemoryPoolListener.onMemoryReserved(pool -> { notifiedPool.set(pool); notifiedBytes.set(pool.getReservedBytes()); })); userPool.reserve(fakeQueryId, "test", 3); assertEquals(notifiedPool.get(), userPool); assertEquals(notifiedBytes.get(), 3L); }
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; }
@Test public void testTaggedAllocations() { QueryId testQuery = new QueryId("test_query"); MemoryPool testPool = new MemoryPool(new MemoryPoolId("test"), new DataSize(1000, BYTE)); testPool.reserve(testQuery, "test_tag", 10); Map<String, Long> allocations = testPool.getTaggedMemoryAllocations().get(testQuery); assertEquals(allocations, ImmutableMap.of("test_tag", 10L)); // free 5 bytes for test_tag testPool.free(testQuery, "test_tag", 5); assertEquals(allocations, ImmutableMap.of("test_tag", 5L)); testPool.reserve(testQuery, "test_tag2", 20); assertEquals(allocations, ImmutableMap.of("test_tag", 5L, "test_tag2", 20L)); // free the remaining 5 bytes for test_tag testPool.free(testQuery, "test_tag", 5); assertEquals(allocations, ImmutableMap.of("test_tag2", 20L)); // free all for test_tag2 testPool.free(testQuery, "test_tag2", 20); assertEquals(testPool.getTaggedMemoryAllocations().size(), 0); }
@Test public void testMemoryFutureCancellation() { setUpCountStarFromOrdersWithJoin(); ListenableFuture future = userPool.reserve(fakeQueryId, "test", TEN_MEGABYTES.toBytes()); assertTrue(!future.isDone()); try { future.cancel(true); fail("cancel should fail"); } catch (UnsupportedOperationException e) { assertEquals(e.getMessage(), "cancellation is not supported"); } userPool.free(fakeQueryId, "test", TEN_MEGABYTES.toBytes()); assertTrue(future.isDone()); }
@Test public void testTrySetZeroBytesFullPool() { LocalMemoryContext localMemoryContext = operatorContext.localUserMemoryContext(); // fill up the pool memoryPool.reserve(new QueryId("test_query"), "test", memoryPool.getFreeBytes()); // try to reserve 0 bytes in the full pool assertTrue(localMemoryContext.trySetBytes(localMemoryContext.getBytes())); }
@Test public void testMoveQuery() { QueryId testQuery = new QueryId("test_query"); MemoryPool pool1 = new MemoryPool(new MemoryPoolId("test"), new DataSize(1000, BYTE)); MemoryPool pool2 = new MemoryPool(new MemoryPoolId("test"), new DataSize(1000, BYTE)); pool1.reserve(testQuery, "test_tag", 10); Map<String, Long> allocations = pool1.getTaggedMemoryAllocations().get(testQuery); assertEquals(allocations, ImmutableMap.of("test_tag", 10L)); pool1.moveQuery(testQuery, pool2); assertNull(pool1.getTaggedMemoryAllocations().get(testQuery)); allocations = pool2.getTaggedMemoryAllocations().get(testQuery); assertEquals(allocations, ImmutableMap.of("test_tag", 10L)); assertEquals(pool1.getFreeBytes(), 1000); assertEquals(pool2.getFreeBytes(), 990); pool2.free(testQuery, "test", 10); assertEquals(pool2.getFreeBytes(), 1000); }
memoryPool.reserve(queryId, "test", reservedMemoryInBytes);
long secondQueryMemory = reservedPool.getMaxBytes() - 1; if (useReservedPool) { assertTrue(reservedPool.reserve(secondQuery, "test", secondQueryMemory).isDone());
public synchronized ListenableFuture<?> reserveSystemMemory(long bytes) { checkArgument(bytes >= 0, "bytes is negative"); ListenableFuture<?> future = systemMemoryPool.reserve(queryId, bytes); systemReserved += bytes; return future; }
public synchronized ListenableFuture<?> reserveMemory(long bytes) { checkArgument(bytes >= 0, "bytes is negative"); if (reserved + bytes > maxMemory) { throw exceededLocalLimit(new DataSize(maxMemory, DataSize.Unit.BYTE).convertToMostSuccinctDataSize()); } ListenableFuture<?> future = memoryPool.reserve(queryId, bytes); reserved += bytes; return future; }
public synchronized void setMemoryPool(MemoryPool pool) { requireNonNull(pool, "pool is null"); if (pool.getId().equals(memoryPool.getId())) { // Don't unblock our tasks and thrash the pools, if this is a no-op return; } MemoryPool originalPool = memoryPool; long originalReserved = reserved; memoryPool = pool; ListenableFuture<?> future = pool.reserve(queryId, reserved); Futures.addCallback(future, new FutureCallback<Object>() { @Override public void onSuccess(Object result) { originalPool.free(queryId, originalReserved); // Unblock all the tasks, if they were waiting for memory, since we're in a new pool. taskContexts.stream().forEach(TaskContext::moreMemoryAvailable); } @Override public void onFailure(Throwable t) { originalPool.free(queryId, originalReserved); // Unblock all the tasks, if they were waiting for memory, since we're in a new pool. taskContexts.stream().forEach(TaskContext::moreMemoryAvailable); } }); }