Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ public HttpRemoteTask(
long numOfPartitionedSources = planFragment.getPartitionedSources().size();
// Currently it supports only when there is one partitioned source.
// TODO. https://github.com/trinodb/trino/issues/15820
this.adaptiveUpdateRequestSizeEnabled = numOfPartitionedSources == 1 ? isRemoteTaskAdaptiveUpdateRequestSizeEnabled(session) : false;
this.adaptiveUpdateRequestSizeEnabled = numOfPartitionedSources == 1 && isRemoteTaskAdaptiveUpdateRequestSizeEnabled(session);
if (numOfPartitionedSources > 1) {
log.debug("%s - There are more than one partitioned sources: numOfPartitionedSources=%s",
taskId, planFragment.getPartitionedSources().size());
Expand Down Expand Up @@ -650,16 +650,16 @@ boolean adjustSplitBatchSize(List<SplitAssignment> splitAssignments, long reques
}
}
if (numSplits != 0) {
newSplitBatchSize = (int) (numSplits * (maxRequestSizeInBytes - requestSizeHeadroomInBytes) / requestSize);
newSplitBatchSize = (int) ((numSplits * (maxRequestSizeInBytes - requestSizeHeadroomInBytes)) / requestSize);
newSplitBatchSize = Math.max(guaranteedSplitsPerRequest, Math.min(maxUnacknowledgedSplits, newSplitBatchSize));
}
if (newSplitBatchSize != currentSplitBatchSize) {
log.debug("%s - Split batch size changed: prevSize=%s, newSize=%s", taskId, currentSplitBatchSize, newSplitBatchSize);
splitBatchSize.set(newSplitBatchSize);
}
// abandon current request and reschedule update if size of request body exceeds requestSizeLimit and splitBatchSize is updated
if (newSplitBatchSize < currentSplitBatchSize) {
log.debug("%s - current taskUpdateRequestJson exceeded limit: %d, currentSplitBatchSize: %d, newSplitBatchSize: %d, reschedule.",
if (numSplits > newSplitBatchSize && requestSize > maxRequestSizeInBytes) {
log.debug("%s - current taskUpdateRequestJson exceeded limit: %d, currentSplitBatchSize: %d, newSplitBatchSize: %d",
taskId, requestSize, currentSplitBatchSize, newSplitBatchSize);
return true; // reschedule needed
}
Expand Down