d-c-manning commented on code in PR #7516:
URL: https://github.com/apache/hbase/pull/7516#discussion_r2593998047
##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java:
##########
@@ -88,23 +103,53 @@ public Configuration getConfiguration() {
return table.getConfiguration();
}
- // will be overridden in test
- protected void internalFlush() {
+ /**
+ * Atomically drains the current buffered mutations and futures under {@link
#lock} and prepares
+ * this mutator to accept a new batch.
+ * <p>
+ * Acquires {@link #lock} and cancels any pending {@link #periodicFlushTask}
to avoid a redundant
Review Comment:
```suggestion
* The {@link #lock} must be acquired before calling this method.
* Cancels any pending {@link #periodicFlushTask} to avoid a redundant
```
##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorImpl.java:
##########
@@ -153,24 +208,61 @@ Stream.<CompletableFuture<Void>>
generate(CompletableFuture::new).limit(mutation
bufferedSize += heapSize;
if (bufferedSize >= writeBufferSize) {
LOG.trace("Flushing because write buffer size {} reached",
writeBufferSize);
- internalFlush();
+ // drain now and send after releasing the lock
+ batch = drainBatch();
} else if (maxMutations > 0 && this.mutations.size() >= maxMutations) {
LOG.trace("Flushing because max mutations {} reached", maxMutations);
- internalFlush();
+ batch = drainBatch();
}
+ } finally {
+ lock.unlock();
+ }
+ // Send outside of lock
+ if (batch != null) {
+ sendBatch(batch);
}
return futures;
}
@Override
- public synchronized void flush() {
- internalFlush();
+ public void flush() {
Review Comment:
Considering that the only difference between `flush` and `close` is the one
statement `closed = true`... is it worth creating a separate method that can be
shared between the two with a parameter for whether to close?
--
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]