You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Currently when a Fluo worker thread executes an observer, the execution flow looks something like the following.
Worker Thread 1 : execute Observer process method for transaction 1
Worker Thread 1 : process method executes get on transaction 1. Wait for this to return
Worker Thread 1 : process method executes get on transaction 1. Wait for this to return
Worker Thread 1 : process method executes get on transaction 1. Wait for this to return
Worker Thread 1 : process method return, transaction is queued for asynchronous commit
While executing get methods above, the worker thread is idle. I am curious if we could design an asynchronous API for observers that allows the worker thread to do other work while a get method is executing.
One possible design would be to have new observer interface like the following.
publicinterfaceAsyncObserver {
/** * This method should immediately return a completable future. When the returned * future completes, the transaction will be committed. */CompletableFuture<Void> process(TransactionBasetx, Bytesrow, Columncol);
}
Below is an example AsyncObserver that is rewrite of the ContentObserver from a Fluo Tour exercise solution. The example below use the getAsync methods proposed in #967
publicstaticclassContentObserverimplementsAsyncObserver
{
privateList<String> tokenize(Stringdocument){//TODO}privateCompletableFuture<Void> adjustCounts(TransactionBasetx, intdelta,
List<String> words) {
List<CompletableFuture<Void>> futures = newArrayList<>();
for (Stringword : newHashSet<>(words)) {
CompletableFuture<Void> future = tx.getsAsync("w:" + word, WORD_COUNT, "0")
.thenApply(Integer::parseInt)
.thenApply(count -> delta + count)
.thenAccept(newCount -> {
if (newCount == 0)
tx.delete("w:" + word, WORD_COUNT);
elsetx.set("w:" + word, WORD_COUNT, newCount + "");
});
futures.add(future);
}
// Return a future that is only complete after all async word operations // are complete. Yuck, why isn't there an allOf method that takes a // Collection?returnCompletableFuture.allOf(futures.toArray(newCompletableFuture[futures.size()]));
}
@OverridepublicCompletableFuture<Void> process(TransactionBasetx, Bytesrow, Columncol) {
CompletableFuture<String> content = tx.getsAsync(row, CONTENT_COL);
CompletableFuture<String> status = tx.getsAsync(row, REF_STATUS_COL);
CompletableFuture<String> processed = tx.getsAsync(row, PROCESSED_COL, "false");
// After all three of the operations complete, then the lambda passed to thenCompose// will either do nothing or return another async operation to compute word counts.returnCompletableFuture.allOf(content, status, processed).thenCompose(v -> {
intdelta = 0;
if (status.get().equals("referenced") && processed.get().equals("false")) {
tx.set(row, PROCESSED_COL, "true");
delta = 1;
}
if (status.get().equals("unreferenced")) {
for (Columnc : newColumn[] {PROCESSED_COL, CONTENT_COL,
REF_COUNT_COL, REF_STATUS_COL})
tx.delete(row, c);
if (processed.get().equals("true")) {
delta = -1;
}
}
if (delta == 0) {
// nothing to do, so return a completed operationreturnCompletableFuture.completedFuture(null);
} else {
// return another async operation to update word countsreturnadjustCounts(tx, delta, tokenize(content.get()));
}
});
}
}
The purpose of having this API would be to increase throughput. To realistically achieve this, Accumulo would need to support async methods as mentioned in #651.
The text was updated successfully, but these errors were encountered:
Currently when a Fluo worker thread executes an observer, the execution flow looks something like the following.
get
on transaction 1. Wait for this to returnget
on transaction 1. Wait for this to returnget
on transaction 1. Wait for this to returnWhile executing get methods above, the worker thread is idle. I am curious if we could design an asynchronous API for observers that allows the worker thread to do other work while a get method is executing.
One possible design would be to have new observer interface like the following.
Below is an example AsyncObserver that is rewrite of the ContentObserver from a Fluo Tour exercise solution. The example below use the getAsync methods proposed in #967
The purpose of having this API would be to increase throughput. To realistically achieve this, Accumulo would need to support async methods as mentioned in #651.
The text was updated successfully, but these errors were encountered: