Skip to content

Commit 022e113

Browse files
authored
Enable show queries to be executed immediately when the available memory in the memoryPool is insufficient (#17507)
1 parent ce103a3 commit 022e113

19 files changed

Lines changed: 430 additions & 115 deletions

File tree

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -129,6 +129,12 @@ public enum ExplainType {
129129

130130
private boolean userQuery = false;
131131

132+
/**
133+
* When true (e.g. SHOW QUERIES), operator and exchange memory may use fallback when pool is
134+
* insufficient. Set from analysis via {@link #setNeedSetHighestPriority(boolean)}.
135+
*/
136+
private boolean needSetHighestPriority = false;
137+
132138
private boolean debug = false;
133139

134140
private Map<NodeRef<Table>, Query> cteQueries = new HashMap<>();
@@ -507,6 +513,14 @@ public void setUserQuery(boolean userQuery) {
507513
this.userQuery = userQuery;
508514
}
509515

516+
public boolean needSetHighestPriority() {
517+
return needSetHighestPriority;
518+
}
519+
520+
public void setNeedSetHighestPriority(boolean needSetHighestPriority) {
521+
this.needSetHighestPriority = needSetHighestPriority;
522+
}
523+
510524
public boolean isDebug() {
511525
return debug;
512526
}

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java

Lines changed: 53 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.iotdb.common.rpc.thrift.TSStatus;
2424
import org.apache.iotdb.commons.client.IClientManager;
2525
import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient;
26+
import org.apache.iotdb.commons.utils.TestOnly;
2627
import org.apache.iotdb.db.queryengine.exception.exchange.GetTsBlockFromClosedOrAbortedChannelException;
2728
import org.apache.iotdb.db.queryengine.execution.driver.DriverContext;
2829
import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelIndex;
@@ -656,7 +657,11 @@ private synchronized ISinkChannel createLocalSinkChannel(
656657
}
657658
queue =
658659
new SharedTsBlockQueue(
659-
localFragmentInstanceId, localPlanNodeId, localMemoryManager, executorService);
660+
localFragmentInstanceId,
661+
localPlanNodeId,
662+
localMemoryManager,
663+
executorService,
664+
instanceContext.isHighestPriority());
660665
}
661666

662667
return new LocalSinkChannel(
@@ -680,7 +685,8 @@ public ISinkChannel createLocalSinkChannelForPipeline(
680685
driverContext.getDriverTaskID().getFragmentInstanceId().toThrift(),
681686
planNodeId,
682687
localMemoryManager,
683-
executorService);
688+
executorService,
689+
driverContext.getFragmentInstanceContext().isHighestPriority());
684690
queue.allowAddingTsBlock();
685691
return new LocalSinkChannel(
686692
queue,
@@ -718,6 +724,7 @@ private ISinkChannel createSinkChannel(
718724
tsBlockSerdeFactory.get(),
719725
new ISinkChannelListenerImpl(
720726
localFragmentInstanceId, instanceContext, instanceContext::failed, cnt),
727+
instanceContext.isHighestPriority(),
721728
mppDataExchangeServiceClientManager);
722729
}
723730

@@ -802,13 +809,32 @@ public ISourceHandle createLocalSourceHandleForPipeline(
802809
context.getDriverTaskID().toString());
803810
}
804811

812+
@TestOnly
805813
public synchronized ISourceHandle createLocalSourceHandleForFragment(
806814
TFragmentInstanceId localFragmentInstanceId,
807815
String localPlanNodeId,
808816
String remotePlanNodeId,
809817
TFragmentInstanceId remoteFragmentInstanceId,
810818
int index,
811819
IMPPDataExchangeManagerCallback<Throwable> onFailureCallback) {
820+
return createLocalSourceHandleForFragment(
821+
localFragmentInstanceId,
822+
localPlanNodeId,
823+
remotePlanNodeId,
824+
remoteFragmentInstanceId,
825+
index,
826+
onFailureCallback,
827+
false);
828+
}
829+
830+
public synchronized ISourceHandle createLocalSourceHandleForFragment(
831+
TFragmentInstanceId localFragmentInstanceId,
832+
String localPlanNodeId,
833+
String remotePlanNodeId,
834+
TFragmentInstanceId remoteFragmentInstanceId,
835+
int index,
836+
IMPPDataExchangeManagerCallback<Throwable> onFailureCallback,
837+
boolean isHighestPriority) {
812838
if (sourceHandles.containsKey(localFragmentInstanceId)
813839
&& sourceHandles.get(localFragmentInstanceId).containsKey(localPlanNodeId)) {
814840
throw new IllegalStateException(
@@ -840,7 +866,11 @@ public synchronized ISourceHandle createLocalSourceHandleForFragment(
840866
}
841867
queue =
842868
new SharedTsBlockQueue(
843-
remoteFragmentInstanceId, remotePlanNodeId, localMemoryManager, executorService);
869+
remoteFragmentInstanceId,
870+
remotePlanNodeId,
871+
localMemoryManager,
872+
executorService,
873+
isHighestPriority);
844874
}
845875
LocalSourceHandle localSourceHandle =
846876
new LocalSourceHandle(
@@ -854,6 +884,7 @@ public synchronized ISourceHandle createLocalSourceHandleForFragment(
854884
return localSourceHandle;
855885
}
856886

887+
@TestOnly
857888
@Override
858889
public ISourceHandle createSourceHandle(
859890
TFragmentInstanceId localFragmentInstanceId,
@@ -862,6 +893,24 @@ public ISourceHandle createSourceHandle(
862893
TEndPoint remoteEndpoint,
863894
TFragmentInstanceId remoteFragmentInstanceId,
864895
IMPPDataExchangeManagerCallback<Throwable> onFailureCallback) {
896+
return createSourceHandle(
897+
localFragmentInstanceId,
898+
localPlanNodeId,
899+
indexOfUpstreamSinkHandle,
900+
remoteEndpoint,
901+
remoteFragmentInstanceId,
902+
onFailureCallback,
903+
false);
904+
}
905+
906+
public ISourceHandle createSourceHandle(
907+
TFragmentInstanceId localFragmentInstanceId,
908+
String localPlanNodeId,
909+
int indexOfUpstreamSinkHandle,
910+
TEndPoint remoteEndpoint,
911+
TFragmentInstanceId remoteFragmentInstanceId,
912+
IMPPDataExchangeManagerCallback<Throwable> onFailureCallback,
913+
boolean isHighestPriority) {
865914
Map<String, ISourceHandle> sourceHandleMap = sourceHandles.get(localFragmentInstanceId);
866915
if (sourceHandleMap != null && sourceHandleMap.containsKey(localPlanNodeId)) {
867916
throw new IllegalStateException(
@@ -891,6 +940,7 @@ public ISourceHandle createSourceHandle(
891940
executorService,
892941
tsBlockSerdeFactory.get(),
893942
new SourceHandleListenerImpl(onFailureCallback),
943+
isHighestPriority,
894944
mppDataExchangeServiceClientManager);
895945
sourceHandles
896946
.computeIfAbsent(localFragmentInstanceId, key -> new ConcurrentHashMap<>())

iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueue.java

Lines changed: 38 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,13 @@
1919

2020
package org.apache.iotdb.db.queryengine.execution.exchange;
2121

22+
import org.apache.iotdb.commons.utils.TestOnly;
2223
import org.apache.iotdb.db.conf.IoTDBDescriptor;
2324
import org.apache.iotdb.db.queryengine.common.FragmentInstanceId;
2425
import org.apache.iotdb.db.queryengine.execution.exchange.sink.LocalSinkChannel;
2526
import org.apache.iotdb.db.queryengine.execution.exchange.source.LocalSourceHandle;
2627
import org.apache.iotdb.db.queryengine.execution.memory.LocalMemoryManager;
28+
import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult;
2729
import org.apache.iotdb.db.utils.CommonUtils;
2830
import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId;
2931

@@ -63,7 +65,7 @@ public class SharedTsBlockQueue {
6365

6466
private long bufferRetainedSizeInBytes = 0L;
6567

66-
private final Queue<TsBlock> queue = new LinkedList<>();
68+
private final Queue<Pair<TsBlock, Long>> queue = new LinkedList<>();
6769

6870
private SettableFuture<Void> blocked = SettableFuture.create();
6971

@@ -83,17 +85,28 @@ public class SharedTsBlockQueue {
8385

8486
private long maxBytesCanReserve =
8587
IoTDBDescriptor.getInstance().getMemoryConfig().getMaxBytesPerFragmentInstance();
88+
private final boolean isHighestPriority;
8689

8790
private volatile Throwable abortedCause = null;
8891

8992
// used for SharedTsBlockQueue listener
9093
private final ExecutorService executorService;
9194

95+
@TestOnly
9296
public SharedTsBlockQueue(
9397
TFragmentInstanceId fragmentInstanceId,
9498
String planNodeId,
9599
LocalMemoryManager localMemoryManager,
96100
ExecutorService executorService) {
101+
this(fragmentInstanceId, planNodeId, localMemoryManager, executorService, false);
102+
}
103+
104+
public SharedTsBlockQueue(
105+
TFragmentInstanceId fragmentInstanceId,
106+
String planNodeId,
107+
LocalMemoryManager localMemoryManager,
108+
ExecutorService executorService,
109+
boolean isHighestPriority) {
97110
this.localFragmentInstanceId =
98111
Validate.notNull(fragmentInstanceId, "fragment instance ID cannot be null");
99112
this.fullFragmentInstanceId =
@@ -102,6 +115,7 @@ public SharedTsBlockQueue(
102115
this.localMemoryManager =
103116
Validate.notNull(localMemoryManager, "local memory manager cannot be null");
104117
this.executorService = Validate.notNull(executorService, "ExecutorService can not be null.");
118+
this.isHighestPriority = isHighestPriority;
105119
}
106120

107121
public boolean hasNoMoreTsBlocks() {
@@ -196,15 +210,18 @@ public TsBlock remove() {
196210
}
197211
throw new IllegalStateException("queue has been destroyed");
198212
}
199-
TsBlock tsBlock = queue.remove();
200-
localMemoryManager
201-
.getQueryPool()
202-
.free(
203-
localFragmentInstanceId.getQueryId(),
204-
fullFragmentInstanceId,
205-
localPlanNodeId,
206-
tsBlock.getSizeInBytes());
207-
bufferRetainedSizeInBytes -= tsBlock.getSizeInBytes();
213+
Pair<TsBlock, Long> tsBlockWithReservedBytes = queue.remove();
214+
long reservedBytes = tsBlockWithReservedBytes.right;
215+
if (reservedBytes > 0) {
216+
localMemoryManager
217+
.getQueryPool()
218+
.free(
219+
localFragmentInstanceId.getQueryId(),
220+
fullFragmentInstanceId,
221+
localPlanNodeId,
222+
reservedBytes);
223+
bufferRetainedSizeInBytes -= reservedBytes;
224+
}
208225
// Every time LocalSourceHandle consumes a TsBlock, it needs to send the event
209226
// to
210227
// corresponding LocalSinkChannel.
@@ -214,7 +231,7 @@ public TsBlock remove() {
214231
if (blocked.isDone() && queue.isEmpty() && !noMoreTsBlocks) {
215232
blocked = SettableFuture.create();
216233
}
217-
return tsBlock;
234+
return tsBlockWithReservedBytes.left;
218235
}
219236

220237
/**
@@ -240,20 +257,22 @@ public ListenableFuture<Void> add(TsBlock tsBlock) {
240257
localFragmentInstanceId.queryId, fullFragmentInstanceId, localPlanNodeId);
241258
alreadyRegistered = true;
242259
}
243-
Pair<ListenableFuture<Void>, Boolean> pair =
260+
MemoryReservationResult reserveResult =
244261
localMemoryManager
245262
.getQueryPool()
246-
.reserve(
263+
.reserveWithPriority(
247264
localFragmentInstanceId.getQueryId(),
248265
fullFragmentInstanceId,
249266
localPlanNodeId,
250267
tsBlock.getSizeInBytes(),
251-
maxBytesCanReserve);
252-
blockedOnMemory = pair.left;
253-
bufferRetainedSizeInBytes += tsBlock.getSizeInBytes();
268+
maxBytesCanReserve,
269+
isHighestPriority);
270+
blockedOnMemory = reserveResult.getFuture();
271+
long reservedBytes = reserveResult.getReservedBytes();
272+
bufferRetainedSizeInBytes += reservedBytes;
254273

255274
// reserve memory failed, we should wait until there is enough memory
256-
if (!Boolean.TRUE.equals(pair.right)) {
275+
if (!reserveResult.isReserveSuccess()) {
257276
SettableFuture<Void> channelBlocked = SettableFuture.create();
258277
blockedOnMemory.addListener(
259278
() -> {
@@ -268,7 +287,7 @@ public ListenableFuture<Void> add(TsBlock tsBlock) {
268287
channelBlocked.set(null);
269288
return;
270289
}
271-
queue.add(tsBlock);
290+
queue.add(new Pair<>(tsBlock, reservedBytes));
272291
if (!blocked.isDone()) {
273292
blocked.set(null);
274293
}
@@ -285,7 +304,7 @@ public ListenableFuture<Void> add(TsBlock tsBlock) {
285304
executorService);
286305
return channelBlocked;
287306
} else { // reserve memory succeeded, add the TsBlock directly
288-
queue.add(tsBlock);
307+
queue.add(new Pair<>(tsBlock, reservedBytes));
289308
if (!blocked.isDone()) {
290309
blocked.set(null);
291310
}

0 commit comments

Comments
 (0)