Skip to content

Commit b4322bb

Browse files
committed
address comment.
1 parent 0be931c commit b4322bb

File tree

1 file changed

+36
-15
lines changed

1 file changed

+36
-15
lines changed

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/cursors/FlatMapPipelinedCursor.java

+36-15
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,7 @@ public class FlatMapPipelinedCursor<T, V> implements RecordCursor<V> {
7878
* should generally be mediated through one of the {@code synchronized} methods.
7979
*/
8080
@Nonnull
81-
private final Queue<PipelineQueueEntry> pipeline;
81+
private final Queue<PipelineQueueEntry<V, T>> pipeline;
8282
/**
8383
* The next value to pull from the outer cursor. This value is cleared out by {@link #close()}, so
8484
* some care should be taken when accessing it. In general, it should be set via one of the {@code synchronized}
@@ -124,7 +124,7 @@ public CompletableFuture<RecordCursorResult<V>> onNext() {
124124
return CompletableFuture.completedFuture(lastResult);
125125
}
126126
return AsyncUtil.whileTrue(this::tryToFillPipeline, getExecutor()).thenApply(vignore -> {
127-
PipelineQueueEntry peeked = peekPipeline();
127+
PipelineQueueEntry<V, T> peeked = peekPipeline();
128128
if (peeked == null) {
129129
throw new CancellationException("cursor closed while iterating");
130130
}
@@ -190,12 +190,12 @@ protected CompletableFuture<Boolean> tryToFillPipeline() {
190190

191191
if (!outerNext.isDone()) {
192192
// Still waiting for outer future. Check back when it has finished.
193-
final PipelineQueueEntry nextEntry = peekPipeline();
193+
final PipelineQueueEntry<V, T> nextEntry = peekPipeline();
194194
if (nextEntry == null) {
195195
return outerNext.thenApply(vignore -> true); // loop back to process outer result
196196
} else {
197197
// keep looping unless we get something from the next entry's inner cursor or the next cursor is ready
198-
final CompletableFuture<PipelineQueueEntry> innerPipelineFuture = nextEntry.getNextInnerPipelineFuture();
198+
final CompletableFuture<PipelineQueueEntry<V, T>> innerPipelineFuture = nextEntry.getNextInnerPipelineFuture();
199199
return CompletableFuture.anyOf(outerNext, innerPipelineFuture).thenApply(vignore ->
200200
!innerPipelineFuture.isDone() || innerPipelineFuture.join().doesNotHaveReturnableResult());
201201
}
@@ -220,12 +220,12 @@ protected CompletableFuture<Boolean> tryToFillPipeline() {
220220
}
221221
final RecordCursor<V> innerCursor = innerCursorFunction.apply(outerValue, innerContinuation);
222222
outerContinuation = outerResult.getContinuation();
223-
addEntryToPipeline(new PipelineQueueEntry(innerCursor, priorOuterContinuation, outerResult, outerCheckValue));
223+
addEntryToPipeline(PipelineQueueEntry.newInstanceWithBackgroundComputationOfFirstResult(innerCursor, priorOuterContinuation, outerResult, outerCheckValue));
224224
outerNextFuture = null; // done with this future, advance outer cursor next time
225225
// keep looping to fill pipeline
226226
} else { // don't have next, and won't ever with this cursor
227227
// Add sentinel to end of pipeline
228-
addEntryToPipeline(new PipelineQueueEntry(null, outerContinuation, outerResult, null));
228+
addEntryToPipeline(PipelineQueueEntry.newSentinel(outerContinuation, outerResult));
229229
outerExhausted = true;
230230
// Wait for next entry, as if pipeline were full
231231
break;
@@ -239,7 +239,7 @@ protected CompletableFuture<Boolean> tryToFillPipeline() {
239239
// 4) A concurrent operation cancelled this cursor and so the pipeline is empty
240240
// The only case where the element should be null is when the cursor has been closed, so return CANCELLED in
241241
// that case.
242-
PipelineQueueEntry peeked = peekPipeline();
242+
PipelineQueueEntry<V, T> peeked = peekPipeline();
243243
if (peeked == null) {
244244
return ALREADY_CANCELLED;
245245
}
@@ -263,45 +263,42 @@ private synchronized CompletableFuture<RecordCursorResult<T>> ensureOuterCursorA
263263
return outerNextFuture;
264264
}
265265

266-
private synchronized void addEntryToPipeline(PipelineQueueEntry pipelineQueueEntry) {
266+
private synchronized void addEntryToPipeline(PipelineQueueEntry<V, T> pipelineQueueEntry) {
267267
if (closed) {
268268
pipelineQueueEntry.close();
269269
}
270-
pipelineQueueEntry.getNextInnerPipelineFuture();
271270
pipeline.add(pipelineQueueEntry);
272271
}
273272

274273
private synchronized boolean continueFillingPipeline() {
275274
return !closed && !outerExhausted && pipeline.size() < pipelineSize;
276275
}
277276

278-
private synchronized PipelineQueueEntry peekPipeline() {
277+
private synchronized PipelineQueueEntry<V, T> peekPipeline() {
279278
// Not a lot in this method, but the underlying queue isn't thread safe so
280279
return pipeline.peek();
281280
}
282281

283-
private class PipelineQueueEntry {
282+
private static class PipelineQueueEntry<V, T> {
284283
final RecordCursor<V> innerCursor;
285284
final RecordCursorContinuation priorOuterContinuation;
286285
final RecordCursorResult<T> outerResult;
287286
final byte[] outerCheckValue;
288287

289288
private CompletableFuture<RecordCursorResult<V>> innerFuture;
290289

291-
public PipelineQueueEntry(RecordCursor<V> innerCursor,
290+
private PipelineQueueEntry(RecordCursor<V> innerCursor,
292291
RecordCursorContinuation priorOuterContinuation,
293292
RecordCursorResult<T> outerResult,
294293
byte[] outerCheckValue) {
295294
this.innerCursor = innerCursor;
296295
this.priorOuterContinuation = priorOuterContinuation;
297296
this.outerResult = outerResult;
298297
this.outerCheckValue = outerCheckValue;
299-
// start calculating the next result in the background.
300-
setInnerFuture();
301298
}
302299

303300
@Nonnull
304-
public CompletableFuture<PipelineQueueEntry> getNextInnerPipelineFuture() {
301+
public CompletableFuture<PipelineQueueEntry<V, T>> getNextInnerPipelineFuture() {
305302
if (innerFuture == null) {
306303
setInnerFuture();
307304
}
@@ -366,6 +363,30 @@ public RecordCursorResult<V> nextResult() {
366363
private Continuation<T, V> toContinuation() {
367364
return new Continuation<>(priorOuterContinuation, outerResult, outerCheckValue, innerFuture.join());
368365
}
366+
367+
@Nonnull
368+
public static <V, T> PipelineQueueEntry<V, T> newInstance(RecordCursor<V> innerCursor,
369+
RecordCursorContinuation priorOuterContinuation,
370+
RecordCursorResult<T> outerResult,
371+
byte[] outerCheckValue) {
372+
return new PipelineQueueEntry<>(innerCursor, priorOuterContinuation, outerResult, outerCheckValue);
373+
}
374+
375+
@Nonnull
376+
public static <V, T> PipelineQueueEntry<V, T> newInstanceWithBackgroundComputationOfFirstResult(RecordCursor<V> innerCursor,
377+
RecordCursorContinuation priorOuterContinuation,
378+
RecordCursorResult<T> outerResult,
379+
byte[] outerCheckValue) {
380+
final var result = newInstance(innerCursor, priorOuterContinuation, outerResult, outerCheckValue);
381+
result.setInnerFuture();
382+
return result;
383+
}
384+
385+
@Nonnull
386+
public static <V, T> PipelineQueueEntry<V, T> newSentinel(RecordCursorContinuation priorOuterContinuation,
387+
RecordCursorResult<T> outerResult) {
388+
return new PipelineQueueEntry<>(null, priorOuterContinuation, outerResult, null);
389+
}
369390
}
370391

371392
private static class Continuation<T, V> implements RecordCursorContinuation {

0 commit comments

Comments
 (0)