-
Notifications
You must be signed in to change notification settings - Fork 3.7k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Emit disk spill and merge buffer utilisation metrics for GroupBy queries #17360
Changes from 1 commit
6fcfcf5
f67ca4d
555f487
4397397
9e71dd1
4b0d3e9
4e03093
ad03d0c
ead93e8
9f2d5a9
cc2d93e
680aebe
6757215
2dd8ed2
2a45d30
2795634
b199d57
518c2a7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -49,4 +49,9 @@ public interface BlockingPool<T> | |||||
* @return count of pending requests | ||||||
*/ | ||||||
long getPendingRequests(); | ||||||
|
||||||
/** | ||||||
* @return number of used buffers from the pool | ||||||
*/ | ||||||
long getUsedBufferCount(); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. BlockingPool is generic. It shouldn't reference buffers.
Suggested change
|
||||||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -61,4 +61,10 @@ public long getPendingRequests() | |
{ | ||
return 0; | ||
} | ||
|
||
@Override | ||
public long getUsedBufferCount() | ||
{ | ||
return 0; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, | ||
* software distributed under the License is distributed on an | ||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
|
||
package org.apache.druid.query.groupby; | ||
|
||
import org.apache.druid.collections.BlockingPool; | ||
import org.apache.druid.guice.annotations.Merging; | ||
import org.apache.druid.query.groupby.epinephelinae.LimitedTemporaryStorage; | ||
|
||
import javax.inject.Inject; | ||
import java.nio.ByteBuffer; | ||
import java.util.Iterator; | ||
import java.util.concurrent.ConcurrentLinkedQueue; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
/** | ||
* Collects stats for group by queries like used merged buffer count, spilled bytes and group by resource acquisition time. | ||
*/ | ||
public class GroupByStatsProvider | ||
{ | ||
private final AtomicLong groupByResourceAcquisitionTimeNs = new AtomicLong(0); | ||
private final AtomicLong groupByResourceAcquisitionCount = new AtomicLong(0); | ||
|
||
private final BlockingPool<ByteBuffer> blockingPool; | ||
private final ConcurrentLinkedQueue<LimitedTemporaryStorage> temporaryStorages; | ||
|
||
@Inject | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: We shouldn't need this |
||
public GroupByStatsProvider(@Merging BlockingPool<ByteBuffer> blockingPool) | ||
{ | ||
this.blockingPool = blockingPool; | ||
this.temporaryStorages = new ConcurrentLinkedQueue<>(); | ||
} | ||
|
||
public synchronized void groupByResourceAcquisitionTimeNs(long delayNs) | ||
{ | ||
groupByResourceAcquisitionTimeNs.addAndGet(delayNs); | ||
groupByResourceAcquisitionCount.incrementAndGet(); | ||
} | ||
|
||
public synchronized long getAndResetGroupByResourceAcquisitionStats() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This doesn't seem correct. We are reporting the average as a metric. A better way would be to report the sum, as well as the count. This will allow a better weighted-average than directly reporting the average from the monitor. While we are amortizing a lot of metrics by emitting it from the monitor, I think it's better to report count and sum separately. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I couldn't find a way wherein I could report the sum and count together (so that any metric aggregation platform could automatically compute the averages). |
||
{ | ||
long average = (groupByResourceAcquisitionTimeNs.get() / groupByResourceAcquisitionCount.get()); | ||
|
||
groupByResourceAcquisitionTimeNs.set(0); | ||
groupByResourceAcquisitionCount.set(0); | ||
|
||
return average; | ||
} | ||
|
||
public long getAcquiredMergeBufferCount() | ||
{ | ||
return blockingPool.getUsedBufferCount(); | ||
} | ||
|
||
public void registerTemporaryStorage(LimitedTemporaryStorage temporaryStorage) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How often does this method need to be called per group by query, and what does this rely upon? I feel that there's an inbuilt expense with tracking the spilled bytes this way. I think it is still fine, but I wonder if a more efficient way would be as follows:
Note: My approach seems more convoluted, so if there isn't much performance downside of the current version, I think its fine as is. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
What is the expense you are referring to? The temporary storage object is added to a queue and cost of adding an element should be in most cases should be O(1).
This approach also requires keeping track of the temporary storages thus incurring similar overhead as the existing implementation. Additionally, it requires incrementing the "running counter" whenever bytes are written out to the disk. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Iterating while calculating the stored bytes.
That is true, but calculating the stored bytes won't require an iteration over the stored memories. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I understand. Given that the monitor will run at a fixed interval (every minute or so), is the overhead significant? |
||
{ | ||
temporaryStorages.add(temporaryStorage); | ||
} | ||
|
||
public long getSpilledBytes() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What if the registered "limited temporary storage" is not closed between two invocations of the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Regardless of this, I think that there should be metric (either this one after repurposing or a new one) which indicates the total size of the spilled data, per query. This will allow the admins to estimate whether the queries need a larger merge buffer and by how much. WDYT? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Yes, that is the intention. It basically is reporting the amount of bytes spilled when the metric is emitted.
Makes sense, but this would be reported at the end of every query? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, the query can update the monitor at the end |
||
{ | ||
long spilledBytes = 0; | ||
|
||
Iterator<LimitedTemporaryStorage> iterator = temporaryStorages.iterator(); | ||
|
||
while (iterator.hasNext()) { | ||
LimitedTemporaryStorage limitedTemporaryStorage = iterator.next(); | ||
|
||
spilledBytes += limitedTemporaryStorage.currentSize(); | ||
|
||
if (limitedTemporaryStorage.isClosed()) { | ||
iterator.remove(); | ||
} | ||
} | ||
|
||
return spilledBytes; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should it be made as a separate monitor? I don't see any downside of merging the QueryCountStatsMonitor with the group by statistics, but at the same time, I don't think there's any benefit to it.
However, merging group-by statistics with "query count stats" feels incongruent to me.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't have strong opinions either, the only reason I merged them was an existing mergeBuffer metric in this monitor. I think a separate monitor would be better due to the added flexibility.