HBASE-29141 Calculate default maxQueueLength call queues correctly#7490
HBASE-29141 Calculate default maxQueueLength call queues correctly#7490apurtell merged 5 commits intoapache:masterfrom
Conversation
4b58ef5 to
86285d4
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
|
@Apache9 , @apurtell , @virajjasani need help in review. |
mnpoonia
left a comment
There was a problem hiding this comment.
@Umeshkumar9414 This is surely a change in good direction.
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
Outdated
Show resolved
Hide resolved
| currentQueueLimit = (int) queueInitArgs[0]; | ||
| queueInitArgs[0] = Math.max((int) queueInitArgs[0], DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT); | ||
| queueInitArgs[0] = Math.min((int) queueInitArgs[0], DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT); | ||
| // queue should neven be initialised with 0 or less length |
There was a problem hiding this comment.
nit typo
| // queue should neven be initialised with 0 or less length | |
| // queue should never be initialised with 0 or less length |
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
Outdated
Show resolved
Hide resolved
| PriorityFunction qosFunction = mock(PriorityFunction.class); | ||
| RWQueueRpcExecutor executor = | ||
| new RWQueueRpcExecutor(testName.getMethodName(), 100, 100, qosFunction, conf, null); | ||
| RWQueueRpcExecutor executor = new RWQueueRpcExecutor(testName.getMethodName(), 100, |
There was a problem hiding this comment.
Could we have a test that validates the queue length limits? Could we validate RpcExecutor#currentQueueLimit directly? Or some other validation that tries to fill the queue, but that may require more work and complexity.
86285d4 to
2835b51
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
|
I tried both the tests (TestAcidGuaranteesWithBasicPolicy and TestAcidGuaranteesWithEagerPolicy) on my local and both are working fine. |
This comment has been minimized.
This comment has been minimized.
82b8ce4 to
a826f1e
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
| .collect(Collectors.groupingBy(Pair::getFirst, Collectors.summingLong(Pair::getSecond))); | ||
| } | ||
|
|
||
| // IMPORTANT: Call this method only ONCE per executor instance. |
There was a problem hiding this comment.
I wonder if we could just enforce this in the code? Only allow it to be called once, and if it is called a second time then throw an "already initialized" error?
| handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); | ||
| int maxPriorityQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH, | ||
| priorityHandlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); | ||
| int maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, -1); |
There was a problem hiding this comment.
can we make the -1 a constant too? something like MAX_CALLQUEUE_LENGTH_UNDEFINED?
| int maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, -1); | |
| int maxQueueLength = conf.getInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, MAX_CALLQUEUE_LENGTH_UNDEFINED); |
| private static final QueueBalancer ONE_QUEUE = val -> 0; | ||
|
|
||
| public static QueueBalancer getBalancer(final String executorName, final Configuration conf, | ||
| protected static QueueBalancer getBalancer(final String executorName, final Configuration conf, |
There was a problem hiding this comment.
this does seem like a useful visibility change, but must it be included here? Changing a public method of a "public" class (LimitedPrivate for COPROC, PHOENIX) is a breaking change. That being said, it's hard to imagine how anyone is using it outside of hbase... and the return class is not LimitedPrivate for COPROC... and I don't see it used in Phoenix... so 🤷♂️
There was a problem hiding this comment.
This method creates the balancer for RPC queues, so I don't think it should be visible outside of the protected visibility.
a826f1e to
394ad47
Compare
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
This comment has been minimized.
|
Both test - TestAcidGuaranteesWithBasicPolicy,TestAcidGuaranteesWithEagerPolicy, are working in local. Seems like flacky or ongoing issue. |
394ad47 to
5d18bb6
Compare
|
🎊 +1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
apurtell
left a comment
There was a problem hiding this comment.
+1, will merge soon unless objection
|
In order to get clean cherry picks back to branch-2 based code, #7620 should be merged first. |
…7490) Co-authored-by: ukumawat <ukumawat@salesforce.com> Signed-off-by: Andrew Purtell <apurtell@apache.org>
…7490) Co-authored-by: ukumawat <ukumawat@salesforce.com> Signed-off-by: Andrew Purtell <apurtell@apache.org>
…7490) Co-authored-by: ukumawat <ukumawat@salesforce.com> Signed-off-by: Andrew Purtell <apurtell@apache.org>
Issues solved -