functioner commented on PR #4349:
URL: https://github.com/apache/hbase/pull/4349#issuecomment-1100748255

   @Apache9 
   I think you have a fair point.
   I think the key question here is how quickly can we retry without taking 
high risk of greatly increasing the load of the server. To answer this 
question, we need to re-examine the goals of using exponential backoff and 
whether our code implementation complies with this design goal.
   So far I haven't found a very detailed discussion on the rationales for 
adding exponential backoff (maybe 
[HBASE-511](https://issues.apache.org/jira/browse/HBASE-511)?) to the current 
HBase client. But from the code comments and Jira issues and 
[wikipedia](https://en.wikipedia.org/wiki/Exponential_backoff), I guess the 
goals of HBase's exponential backoff may include:
   1. **When the server is not available**, the client should avoid to 
frequently resend the requests without getting substantial responses. The 
exponential backoff is a trade-off between waiting for the server startup and 
resending the request.
   2. **When the clients are sending too many requests to the server**, a 
failed request is signal to let the clients slow down, so that the clients' 
request rates reach an acceptable equilibrium.
   3. **When the server tries to handle the request and encounters some 
error**, the server may want the client to wait a little more time so that the 
server can have enough time to tolerate the error. The exponential backoff is a 
trade-off between waiting for the fault tolerance and resending the request.
   
   In general, I feel that if we can find the reason for the request error 
(e.g., server unavailable), we can **do the fine-grained exponential backoff 
separately, based on the error**. For example, I find that in the current 
master branch of Hadoop and Kafka, the client tries to connect to the server 
with exponential backoff pause time, and after building the connection, the 
client uses another exponential backoff retry loop to handle the potential 
exception. They do not share the same exponential backoff retry loop because 
they are handling different problems. **Blindly handling those errors in one 
single exponential backoff retry loop may result in unnecessary long pause 
time, without solving any problem.**
   
   Specifically, in our case, the HMaster checks the initialization before 
handling the clients' requests. My client gets these exceptions, waits for this 
initialization, using the exponential backoff.
   
https://github.com/apache/hbase/blob/2622fa05a6aa9e0b57d9d15cf243af11927e9a2d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java#L3091-L3100
   And after the successful initialization, the server can handle my clients' 
request but somehow gets some error in:
   
https://github.com/apache/hbase/blob/2622fa05a6aa9e0b57d9d15cf243af11927e9a2d/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/store/region/RegionProcedureStore.java#L360-L372
   The `UncheckedIOException` is passed to my client, and then unfortunately, 
the accumulated retry number (due to multiple previous `PleaseHoldException`) 
results in a very long pause time.
   
   As you suggested, I can "set retry number to 1". And I think it should be 
automatically done by the client. The `PleaseHoldException` and the 
`UncheckedIOException` from `update` operation represents two different reasons 
for exponential backoff. `PleaseHoldException` asks the client to wait for the 
server initialization in a exponential backoff retry loop. And after that, to 
handle the potential errors of processing client requests, we should start 
another exponential backoff retry loop, i.e., "set retry number to 1".
   
   WDYT?
   If you are convinced, do you have suggestions about enforcing this kind of 
logic in the code? I feel we may need to add some flags to indicate whether we 
are waiting for server initialization or handling errors in request processing. 
And then reset the retry number to 1 when the flag changes.
   
   Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to