This is an automated email from the ASF dual-hosted git repository.
ctubbsii pushed a commit to branch 1.10
in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/1.10 by this push:
new da2b7ed883 Fix scan thread resource starvation (#3226)
da2b7ed883 is described below
commit da2b7ed883a8e99c733fc557032b1a45544cddf1
Author: Daniel Roberts <[email protected]>
AuthorDate: Mon Mar 6 13:19:35 2023 -0500
Fix scan thread resource starvation (#3226)
This commit switches the filePermits semaphore to a "fair" semaphore
which ensures that threads requesting reserved readers are handled in a
FIFO queue vs ending up in a starvation state for resources.
---
.../main/java/org/apache/accumulo/tserver/FileManager.java | 11 ++++++-----
1 file changed, 6 insertions(+), 5 deletions(-)
diff --git
a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
index cfb97a6027..c36fe3a8e4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
@@ -178,7 +178,8 @@ public class FileManager {
this.indexCache = indexCache;
this.fileLenCache = fileLenCache;
- this.filePermits = new Semaphore(maxOpen, false);
+ // Creates a fair semaphore to ensure thread starvation doesn't occur
+ this.filePermits = new Semaphore(maxOpen, true);
this.maxOpen = maxOpen;
this.fs = fs;
@@ -295,17 +296,17 @@ public class FileManager {
long waitTime = System.currentTimeMillis() - start;
if (waitTime >= slowFilePermitMillis) {
- log.info("Slow file permits request: {} ms, files requested: {},
tablet: {}", waitTime,
- files.size(), tablet);
+ log.warn("Slow file permits request: {} ms, files requested: {}, "
+ + "max open files: {}, tablet: {}", waitTime, files.size(),
maxOpen, tablet);
}
}
- // now that the we are past the semaphore, we have the authority
+ // now that we are past the semaphore, we have the authority
// to open files.size() files
// determine what work needs to be done in sync block
// but do the work of opening and closing files outside
- // a synch block
+ // the block
synchronized (this) {
filesToOpen = takeOpenFiles(files, readersReserved);