From 6e302206f43a40b44982ff2e2747a6dff579d5ff Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sun, 12 May 2024 17:56:56 +1000 Subject: [PATCH 1/9] Add a proof-of-concept for "Observer-like" batch loading **Note**: This commit, as-is, is not (yet) intended for merge. It is created to provide a proof-of-concept and gauge interest as polishing/testing this requires a non-trivial amount of effort. Motivation ========== The current DataLoader mechanism completes the corresponding `CompletableFuture` for a given key when the corresponding value is returned. However, DataLoader's `BatchLoader` assumes that the underlying batch function can only return all of its requested items at once (as an example, a SQL database query). However, the batch function may be a service that can return items progressively using a subscription-like architecture. Some examples include: - Project Reactor's [Subscriber](https://www.reactive-streams.org/reactive-streams-1.0.4-javadoc/org/reactivestreams/Subscriber.html). - gRPC's [StreamObserver](https://grpc.github.io/grpc-java/javadoc/io/grpc/stub/StreamObserver.html). - RX Java's [Flowable](https://reactivex.io/RxJava/3.x/javadoc/io/reactivex/rxjava3/core/Flowable.html). Streaming results in this fashion offers several advantages: - Certain values may be returned earlier than others (for example, the batch function may have cached values it can return early). - Memory load is lessened on the batch function (which may be an external service), as it does not need to keep hold of the retrieved values before it can send them out at once. - We are able to save the need to stream individual error values by providing an `onError` function to terminate the stream early. Proposal ======== We provide two new `BatchLoader`s and support for them in `java-dataloader`: - `ObserverBatchLoader`, with a load function that accepts: - a list of keys. - a `BatchObserver` intended as a delegate for publisher-like structures found in Project Reactor and Rx Java. This obviates the need to depend on external libraries. - `MappedObserverBatchLoader`, similar to `ObserverBatchLoader` but with an `onNext` that accepts a key _and_ value (to allow for early termination of streams without needing to process `null`s). - `*WithContext` variants for the above. The key value-add is that the implementation of `BatchObserver` (provided to the load functions) will immediately complete the queued future for a given key when `onNext` is called with a value. This means that if we have a batch function that can deliver values progressively, we can continue evaluating the query as the values arrive. As an arbitrary example, let's have a batch function that serves both the reporter and project fields on a Jira issue: ```graphql query { issue { project { issueTypes { ... } } reporter { ... } } } ``` If the batch function can return a `project` immediately but is delayed in when it can `reporter`, then our batch loader can return `project` and start evaluating the `issueTypes` immediately while we load the `reporter` in parallel. This would provide a more performant query evaluation. As mentioned above, this is not in a state to be merged - this is intended to gauge whether this is something the maintainers would be interested in owning. Should this be the case, the author is willing to test/polish this pull request so that it may be merged. --- .../java/org/dataloader/BatchObserver.java | 33 +++ .../java/org/dataloader/DataLoaderHelper.java | 267 +++++++++++++++++- .../org/dataloader/MappedBatchObserver.java | 34 +++ .../dataloader/MappedObserverBatchLoader.java | 17 ++ .../MappedObserverBatchLoaderWithContext.java | 10 + .../org/dataloader/ObserverBatchLoader.java | 19 ++ .../ObserverBatchLoaderWithContext.java | 10 + .../scheduler/BatchLoaderScheduler.java | 21 ++ src/test/java/ReadmeExamples.java | 6 + ...taLoaderMappedObserverBatchLoaderTest.java | 106 +++++++ .../DataLoaderObserverBatchLoaderTest.java | 108 +++++++ .../scheduler/BatchLoaderSchedulerTest.java | 20 ++ 12 files changed, 644 insertions(+), 7 deletions(-) create mode 100644 src/main/java/org/dataloader/BatchObserver.java create mode 100644 src/main/java/org/dataloader/MappedBatchObserver.java create mode 100644 src/main/java/org/dataloader/MappedObserverBatchLoader.java create mode 100644 src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java create mode 100644 src/main/java/org/dataloader/ObserverBatchLoader.java create mode 100644 src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java create mode 100644 src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java create mode 100644 src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java diff --git a/src/main/java/org/dataloader/BatchObserver.java b/src/main/java/org/dataloader/BatchObserver.java new file mode 100644 index 0000000..14ef051 --- /dev/null +++ b/src/main/java/org/dataloader/BatchObserver.java @@ -0,0 +1,33 @@ +package org.dataloader; + +/** + * A interface intended as a delegate for other Observer-like classes used in other libraries, to be invoked by the calling + * {@link ObserverBatchLoader}. + *

+ * Some examples include: + *

+ * @param the value type of the {@link ObserverBatchLoader} + */ +public interface BatchObserver { + + /** + * To be called by the {@link ObserverBatchLoader} to load a new value. + */ + void onNext(V value); + + /** + * To be called by the {@link ObserverBatchLoader} to indicate all values have been successfully processed. + * This {@link BatchObserver} should not have any method invoked after this is called. + */ + void onCompleted(); + + /** + * To be called by the {@link ObserverBatchLoader} to indicate an unrecoverable error has been encountered. + * This {@link BatchObserver} should not have any method invoked after this is called. + */ + void onError(Throwable e); +} diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index d934de2..47d2d35 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -15,6 +15,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -241,10 +242,14 @@ private CompletableFuture> sliceIntoBatchesOfBatches(List keys, List< @SuppressWarnings("unchecked") private CompletableFuture> dispatchQueueBatch(List keys, List callContexts, List> queuedFutures) { stats.incrementBatchLoadCountBy(keys.size(), new IncrementBatchLoadCountByStatisticsContext<>(keys, callContexts)); - CompletableFuture> batchLoad = invokeLoader(keys, callContexts, loaderOptions.cachingEnabled()); + CompletableFuture> batchLoad = invokeLoader(keys, callContexts, queuedFutures, loaderOptions.cachingEnabled()); return batchLoad .thenApply(values -> { assertResultSize(keys, values); + if (isObserverLoader() || isMapObserverLoader()) { + // We have already completed the queued futures by the time the overall batchLoad future has completed. + return values; + } List clearCacheKeys = new ArrayList<>(); for (int idx = 0; idx < queuedFutures.size(); idx++) { @@ -342,14 +347,15 @@ private CompletableFuture queueOrInvokeLoader(K key, Object loadContext, bool CompletableFuture invokeLoaderImmediately(K key, Object keyContext, boolean cachingEnabled) { List keys = singletonList(key); List keyContexts = singletonList(keyContext); - return invokeLoader(keys, keyContexts, cachingEnabled) + List> queuedFutures = singletonList(new CompletableFuture<>()); + return invokeLoader(keys, keyContexts, queuedFutures, cachingEnabled) .thenApply(list -> list.get(0)) .toCompletableFuture(); } - CompletableFuture> invokeLoader(List keys, List keyContexts, boolean cachingEnabled) { + CompletableFuture> invokeLoader(List keys, List keyContexts, List> queuedFutures, boolean cachingEnabled) { if (!cachingEnabled) { - return invokeLoader(keys, keyContexts); + return invokeLoader(keys, keyContexts, queuedFutures); } CompletableFuture>> cacheCallCF = getFromValueCache(keys); return cacheCallCF.thenCompose(cachedValues -> { @@ -360,6 +366,7 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, List missedKeyIndexes = new ArrayList<>(); List missedKeys = new ArrayList<>(); List missedKeyContexts = new ArrayList<>(); + List> missedQueuedFutures = new ArrayList<>(); // if they return a ValueCachingNotSupported exception then we insert this special marker value, and it // means it's a total miss, we need to get all these keys via the batch loader @@ -369,6 +376,7 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, missedKeyIndexes.add(i); missedKeys.add(keys.get(i)); missedKeyContexts.add(keyContexts.get(i)); + missedQueuedFutures.add(queuedFutures.get(i)); } } else { assertState(keys.size() == cachedValues.size(), () -> "The size of the cached values MUST be the same size as the key list"); @@ -393,7 +401,7 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, // we missed some keys from cache, so send them to the batch loader // and then fill in their values // - CompletableFuture> batchLoad = invokeLoader(missedKeys, missedKeyContexts); + CompletableFuture> batchLoad = invokeLoader(missedKeys, missedKeyContexts, missedQueuedFutures); return batchLoad.thenCompose(missedValues -> { assertResultSize(missedKeys, missedValues); @@ -412,8 +420,7 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, }); } - - CompletableFuture> invokeLoader(List keys, List keyContexts) { + CompletableFuture> invokeLoader(List keys, List keyContexts, List> queuedFutures) { CompletableFuture> batchLoad; try { Object context = loaderOptions.getBatchLoaderContextProvider().getContext(); @@ -421,6 +428,10 @@ CompletableFuture> invokeLoader(List keys, List keyContexts) .context(context).keyContexts(keys, keyContexts).build(); if (isMapLoader()) { batchLoad = invokeMapBatchLoader(keys, environment); + } else if (isObserverLoader()) { + batchLoad = invokeObserverBatchLoader(keys, keyContexts, queuedFutures, environment); + } else if (isMapObserverLoader()) { + batchLoad = invokeMappedObserverBatchLoader(keys, keyContexts, queuedFutures, environment); } else { batchLoad = invokeListBatchLoader(keys, environment); } @@ -492,10 +503,68 @@ private CompletableFuture> invokeMapBatchLoader(List keys, BatchLoade }); } + private CompletableFuture> invokeObserverBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + CompletableFuture> loadResult = new CompletableFuture<>(); + BatchObserver observer = new BatchObserverImpl(loadResult, keys, keyContexts, queuedFutures); + + BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); + if (batchLoadFunction instanceof ObserverBatchLoaderWithContext) { + ObserverBatchLoaderWithContext loadFunction = (ObserverBatchLoaderWithContext) batchLoadFunction; + if (batchLoaderScheduler != null) { + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer, environment); + batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); + } else { + loadFunction.load(keys, observer, environment); + } + } else { + ObserverBatchLoader loadFunction = (ObserverBatchLoader) batchLoadFunction; + if (batchLoaderScheduler != null) { + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer); + batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); + } else { + loadFunction.load(keys, observer); + } + } + return loadResult; + } + + private CompletableFuture> invokeMappedObserverBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + CompletableFuture> loadResult = new CompletableFuture<>(); + MappedBatchObserver observer = new MappedBatchObserverImpl(loadResult, keys, keyContexts, queuedFutures); + + BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); + if (batchLoadFunction instanceof MappedObserverBatchLoaderWithContext) { + MappedObserverBatchLoaderWithContext loadFunction = (MappedObserverBatchLoaderWithContext) batchLoadFunction; + if (batchLoaderScheduler != null) { + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer, environment); + batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); + } else { + loadFunction.load(keys, observer, environment); + } + } else { + MappedObserverBatchLoader loadFunction = (MappedObserverBatchLoader) batchLoadFunction; + if (batchLoaderScheduler != null) { + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer); + batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); + } else { + loadFunction.load(keys, observer); + } + } + return loadResult; + } + private boolean isMapLoader() { return batchLoadFunction instanceof MappedBatchLoader || batchLoadFunction instanceof MappedBatchLoaderWithContext; } + private boolean isObserverLoader() { + return batchLoadFunction instanceof ObserverBatchLoader; + } + + private boolean isMapObserverLoader() { + return batchLoadFunction instanceof MappedObserverBatchLoader; + } + int dispatchDepth() { synchronized (dataLoader) { return loaderQueue.size(); @@ -546,4 +615,188 @@ private CompletableFuture> setToValueCache(List assembledValues, List private static DispatchResult emptyDispatchResult() { return (DispatchResult) EMPTY_DISPATCH_RESULT; } + + private class BatchObserverImpl implements BatchObserver { + private final CompletableFuture> valuesFuture; + private final List keys; + private final List callContexts; + private final List> queuedFutures; + + private final List clearCacheKeys = new ArrayList<>(); + private final List completedValues = new ArrayList<>(); + private int idx = 0; + private boolean onErrorCalled = false; + private boolean onCompletedCalled = false; + + private BatchObserverImpl( + CompletableFuture> valuesFuture, + List keys, + List callContexts, + List> queuedFutures + ) { + this.valuesFuture = valuesFuture; + this.keys = keys; + this.callContexts = callContexts; + this.queuedFutures = queuedFutures; + } + + @Override + public void onNext(V value) { + assert !onErrorCalled && !onCompletedCalled; + + K key = keys.get(idx); + Object callContext = callContexts.get(idx); + CompletableFuture future = queuedFutures.get(idx); + if (value instanceof Throwable) { + stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); + future.completeExceptionally((Throwable) value); + clearCacheKeys.add(keys.get(idx)); + } else if (value instanceof Try) { + // we allow the batch loader to return a Try so we can better represent a computation + // that might have worked or not. + Try tryValue = (Try) value; + if (tryValue.isSuccess()) { + future.complete(tryValue.get()); + } else { + stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); + future.completeExceptionally(tryValue.getThrowable()); + clearCacheKeys.add(keys.get(idx)); + } + } else { + future.complete(value); + } + + completedValues.add(value); + idx++; + } + + @Override + public void onCompleted() { + assert !onErrorCalled; + onCompletedCalled = true; + + assertResultSize(keys, completedValues); + + possiblyClearCacheEntriesOnExceptions(clearCacheKeys); + valuesFuture.complete(completedValues); + } + + @Override + public void onError(Throwable ex) { + assert !onCompletedCalled; + onErrorCalled = true; + + stats.incrementBatchLoadExceptionCount(new IncrementBatchLoadExceptionCountStatisticsContext<>(keys, callContexts)); + if (ex instanceof CompletionException) { + ex = ex.getCause(); + } + // Set the remaining keys to the exception. + for (int i = idx; i < queuedFutures.size(); i++) { + K key = keys.get(i); + CompletableFuture future = queuedFutures.get(i); + future.completeExceptionally(ex); + // clear any cached view of this key because they all failed + dataLoader.clear(key); + } + } + } + + private class MappedBatchObserverImpl implements MappedBatchObserver { + private final CompletableFuture> valuesFuture; + private final List keys; + private final List callContexts; + private final List> queuedFutures; + private final Map callContextByKey; + private final Map> queuedFutureByKey; + + private final List clearCacheKeys = new ArrayList<>(); + private final Map completedValuesByKey = new HashMap<>(); + private boolean onErrorCalled = false; + private boolean onCompletedCalled = false; + + private MappedBatchObserverImpl( + CompletableFuture> valuesFuture, + List keys, + List callContexts, + List> queuedFutures + ) { + this.valuesFuture = valuesFuture; + this.keys = keys; + this.callContexts = callContexts; + this.queuedFutures = queuedFutures; + + this.callContextByKey = new HashMap<>(); + this.queuedFutureByKey = new HashMap<>(); + for (int idx = 0; idx < queuedFutures.size(); idx++) { + K key = keys.get(idx); + Object callContext = callContexts.get(idx); + CompletableFuture queuedFuture = queuedFutures.get(idx); + callContextByKey.put(key, callContext); + queuedFutureByKey.put(key, queuedFuture); + } + } + + @Override + public void onNext(K key, V value) { + assert !onErrorCalled && !onCompletedCalled; + + Object callContext = callContextByKey.get(key); + CompletableFuture future = queuedFutureByKey.get(key); + if (value instanceof Throwable) { + stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); + future.completeExceptionally((Throwable) value); + clearCacheKeys.add(key); + } else if (value instanceof Try) { + // we allow the batch loader to return a Try so we can better represent a computation + // that might have worked or not. + Try tryValue = (Try) value; + if (tryValue.isSuccess()) { + future.complete(tryValue.get()); + } else { + stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); + future.completeExceptionally(tryValue.getThrowable()); + clearCacheKeys.add(key); + } + } else { + future.complete(value); + } + + completedValuesByKey.put(key, value); + } + + @Override + public void onCompleted() { + assert !onErrorCalled; + onCompletedCalled = true; + + possiblyClearCacheEntriesOnExceptions(clearCacheKeys); + List values = new ArrayList<>(keys.size()); + for (K key : keys) { + V value = completedValuesByKey.get(key); + values.add(value); + } + valuesFuture.complete(values); + } + + @Override + public void onError(Throwable ex) { + assert !onCompletedCalled; + onErrorCalled = true; + + stats.incrementBatchLoadExceptionCount(new IncrementBatchLoadExceptionCountStatisticsContext<>(keys, callContexts)); + if (ex instanceof CompletionException) { + ex = ex.getCause(); + } + // Complete the futures for the remaining keys with the exception. + for (int idx = 0; idx < queuedFutures.size(); idx++) { + K key = keys.get(idx); + CompletableFuture future = queuedFutureByKey.get(key); + if (!completedValuesByKey.containsKey(key)) { + future.completeExceptionally(ex); + // clear any cached view of this key because they all failed + dataLoader.clear(key); + } + } + } + } } diff --git a/src/main/java/org/dataloader/MappedBatchObserver.java b/src/main/java/org/dataloader/MappedBatchObserver.java new file mode 100644 index 0000000..59a0f73 --- /dev/null +++ b/src/main/java/org/dataloader/MappedBatchObserver.java @@ -0,0 +1,34 @@ +package org.dataloader; + +/** + * A interface intended as a delegate for other Observer-like classes used in other libraries, to be invoked by the calling + * {@link MappedObserverBatchLoader}. + *

+ * Some examples include: + *

+ * @param the key type of the calling {@link MappedObserverBatchLoader}. + * @param the value type of the calling {@link MappedObserverBatchLoader}. + */ +public interface MappedBatchObserver { + + /** + * To be called by the {@link MappedObserverBatchLoader} to process a new key/value pair. + */ + void onNext(K key, V value); + + /** + * To be called by the {@link MappedObserverBatchLoader} to indicate all values have been successfully processed. + * This {@link MappedBatchObserver} should not have any method invoked after this method is called. + */ + void onCompleted(); + + /** + * To be called by the {@link MappedObserverBatchLoader} to indicate an unrecoverable error has been encountered. + * This {@link MappedBatchObserver} should not have any method invoked after this method is called. + */ + void onError(Throwable e); +} diff --git a/src/main/java/org/dataloader/MappedObserverBatchLoader.java b/src/main/java/org/dataloader/MappedObserverBatchLoader.java new file mode 100644 index 0000000..d82ec75 --- /dev/null +++ b/src/main/java/org/dataloader/MappedObserverBatchLoader.java @@ -0,0 +1,17 @@ +package org.dataloader; + +import java.util.List; + +/** + * A function that is invoked for batch loading a stream of data values indicated by the provided list of keys. + *

+ * The function will call the provided {@link MappedBatchObserver} to process the key/value pairs it has retrieved to allow + * the future returned by {@link DataLoader#load(Object)} to complete as soon as the individual value is available + * (rather than when all values have been retrieved). + * + * @param type parameter indicating the type of keys to use for data load requests. + * @param type parameter indicating the type of values returned + */ +public interface MappedObserverBatchLoader { + void load(List keys, MappedBatchObserver observer); +} diff --git a/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java b/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java new file mode 100644 index 0000000..6619198 --- /dev/null +++ b/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java @@ -0,0 +1,10 @@ +package org.dataloader; + +import java.util.List; + +/** + * A {@link MappedObserverBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + */ +public interface MappedObserverBatchLoaderWithContext { + void load(List keys, MappedBatchObserver observer, BatchLoaderEnvironment environment); +} diff --git a/src/main/java/org/dataloader/ObserverBatchLoader.java b/src/main/java/org/dataloader/ObserverBatchLoader.java new file mode 100644 index 0000000..0c481f9 --- /dev/null +++ b/src/main/java/org/dataloader/ObserverBatchLoader.java @@ -0,0 +1,19 @@ +package org.dataloader; + +import java.util.List; + +/** + * A function that is invoked for batch loading a stream of data values indicated by the provided list of keys. + *

+ * The function will call the provided {@link BatchObserver} to process the values it has retrieved to allow + * the future returned by {@link DataLoader#load(Object)} to complete as soon as the individual value is available + * (rather than when all values have been retrieved). + *

+ * It is required that values be returned in the same order as the keys provided. + * + * @param type parameter indicating the type of keys to use for data load requests. + * @param type parameter indicating the type of values returned + */ +public interface ObserverBatchLoader { + void load(List keys, BatchObserver observer); +} diff --git a/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java b/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java new file mode 100644 index 0000000..14a3dd1 --- /dev/null +++ b/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java @@ -0,0 +1,10 @@ +package org.dataloader; + +import java.util.List; + +/** + * An {@link ObserverBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + */ +public interface ObserverBatchLoaderWithContext { + void load(List keys, BatchObserver observer, BatchLoaderEnvironment environment); +} diff --git a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java index 7cddd54..5b88d2c 100644 --- a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java +++ b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java @@ -5,6 +5,8 @@ import org.dataloader.DataLoader; import org.dataloader.DataLoaderOptions; import org.dataloader.MappedBatchLoader; +import org.dataloader.MappedObserverBatchLoader; +import org.dataloader.ObserverBatchLoader; import java.util.List; import java.util.Map; @@ -42,6 +44,13 @@ interface ScheduledMappedBatchLoaderCall { CompletionStage> invoke(); } + /** + * This represents a callback that will invoke a {@link ObserverBatchLoader} or {@link MappedObserverBatchLoader} function under the covers + */ + interface ScheduledObserverBatchLoaderCall { + void invoke(); + } + /** * This is called to schedule a {@link BatchLoader} call. * @@ -71,4 +80,16 @@ interface ScheduledMappedBatchLoaderCall { * @return a promise to the values that come from the {@link BatchLoader} */ CompletionStage> scheduleMappedBatchLoader(ScheduledMappedBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); + + /** + * This is called to schedule a {@link ObserverBatchLoader} call. + * + * @param scheduledCall the callback that needs to be invoked to allow the {@link ObserverBatchLoader} to proceed. + * @param keys this is the list of keys that will be passed to the {@link ObserverBatchLoader}. + * This is provided only for informative reasons and, you can't change the keys that are used + * @param environment this is the {@link BatchLoaderEnvironment} in place, + * which can be null if it's a simple {@link ObserverBatchLoader} call + * @param the key type + */ + void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); } diff --git a/src/test/java/ReadmeExamples.java b/src/test/java/ReadmeExamples.java index d25dfa7..df733ed 100644 --- a/src/test/java/ReadmeExamples.java +++ b/src/test/java/ReadmeExamples.java @@ -304,6 +304,12 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp return scheduledCall.invoke(); }).thenCompose(Function.identity()); } + + @Override + public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + snooze(10); + scheduledCall.invoke(); + } }; } diff --git a/src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java new file mode 100644 index 0000000..e6f1168 --- /dev/null +++ b/src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java @@ -0,0 +1,106 @@ +package org.dataloader; + +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.util.Arrays.asList; +import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toMap; +import static org.awaitility.Awaitility.await; +import static org.dataloader.DataLoaderFactory.mkDataLoader; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +public class DataLoaderMappedObserverBatchLoaderTest { + + @Test + public void should_Build_a_really_really_simple_data_loader() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + CompletionStage future1 = identityLoader.load(1); + + future1.thenAccept(value -> { + assertThat(value, equalTo(1)); + success.set(true); + }); + identityLoader.dispatch(); + await().untilAtomic(success, is(true)); + } + + @Test + public void should_Support_loading_multiple_keys_in_one_call() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); + futureAll.thenAccept(promisedValues -> { + assertThat(promisedValues.size(), is(2)); + success.set(true); + }); + identityLoader.dispatch(); + await().untilAtomic(success, is(true)); + assertThat(futureAll.toCompletableFuture().join(), equalTo(asList(1, 2))); + } + + @Test + public void simple_dataloader() { + DataLoader loader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + loader.load("A"); + loader.load("B"); + loader.loadMany(asList("C", "D")); + + List results = loader.dispatchAndJoin(); + + assertThat(results.size(), equalTo(4)); + assertThat(results, equalTo(asList("A", "B", "C", "D"))); + } + + @Test + public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { + DataLoader identityLoader = mkDataLoader(keysAsValues(), new DataLoaderOptions()); + + CompletableFuture future1 = identityLoader.load(1); + CompletableFuture future2 = identityLoader.load(2); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo(1)); + assertThat(future2.get(), equalTo(2)); + } + + // A simple wrapper class intended as a proof external libraries can leverage this. + private static class Publisher { + + private final MappedBatchObserver delegate; + private Publisher(MappedBatchObserver delegate) { this.delegate = delegate; } + void onNext(Map.Entry entry) { delegate.onNext(entry.getKey(), entry.getValue()); } + void onCompleted() { delegate.onCompleted(); } + void onError(Throwable e) { delegate.onError(e); } + // Mock 'subscribe' methods to simulate what would happen in the real thing. + void subscribe(Map valueByKey) { + valueByKey.entrySet().forEach(this::onNext); + this.onCompleted(); + } + void subscribe(Map valueByKey, Throwable e) { + valueByKey.entrySet().forEach(this::onNext); + this.onError(e); + } + } + + private static MappedObserverBatchLoader keysAsValues() { + return (keys, observer) -> { + Publisher publisher = new Publisher<>(observer); + Map valueByKey = keys.stream().collect(toMap(identity(), identity())); + publisher.subscribe(valueByKey); + }; + } +} diff --git a/src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java new file mode 100644 index 0000000..eaeef8f --- /dev/null +++ b/src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java @@ -0,0 +1,108 @@ +package org.dataloader; + +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.util.Arrays.asList; +import static org.awaitility.Awaitility.await; +import static org.dataloader.DataLoaderFactory.mkDataLoader; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +public class DataLoaderObserverBatchLoaderTest { + + @Test + public void should_Build_a_really_really_simple_data_loader() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + CompletionStage future1 = identityLoader.load(1); + + future1.thenAccept(value -> { + assertThat(value, equalTo(1)); + success.set(true); + }); + identityLoader.dispatch(); + await().untilAtomic(success, is(true)); + } + + @Test + public void should_Support_loading_multiple_keys_in_one_call() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); + futureAll.thenAccept(promisedValues -> { + assertThat(promisedValues.size(), is(2)); + success.set(true); + }); + identityLoader.dispatch(); + await().untilAtomic(success, is(true)); + assertThat(futureAll.toCompletableFuture().join(), equalTo(asList(1, 2))); + } + + @Test + public void simple_dataloader() { + DataLoader loader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + + loader.load("A"); + loader.load("B"); + loader.loadMany(asList("C", "D")); + + List results = loader.dispatchAndJoin(); + + assertThat(results.size(), equalTo(4)); + assertThat(results, equalTo(asList("A", "B", "C", "D"))); + } + + @Test + public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { + DataLoader identityLoader = mkDataLoader(keysAsValues(), new DataLoaderOptions()); + + CompletableFuture future1 = identityLoader.load(1); + CompletableFuture future2 = identityLoader.load(2); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo(1)); + assertThat(future2.get(), equalTo(2)); + } + + // A simple wrapper class intended as a proof external libraries can leverage this. + private static class Publisher { + private final BatchObserver delegate; + private Publisher(BatchObserver delegate) { this.delegate = delegate; } + void onNext(V value) { delegate.onNext(value); } + void onCompleted() { delegate.onCompleted(); } + void onError(Throwable e) { delegate.onError(e); } + // Mock 'subscribe' methods to simulate what would happen in the real thing. + void subscribe(List values) { + values.forEach(this::onNext); + this.onCompleted(); + } + void subscribe(List values, Throwable e) { + values.forEach(this::onNext); + this.onError(e); + } + } + + private static ObserverBatchLoader keysAsValues() { + return (keys, observer) -> { + Publisher publisher = new Publisher<>(observer); + publisher.subscribe(keys); + }; + } + + private static ObserverBatchLoader keysWithValuesAndException(List values, Throwable e) { + return (keys, observer) -> { + Publisher publisher = new Publisher<>(observer); + publisher.subscribe(values, e); + }; + } +} diff --git a/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java b/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java index beb7c18..b77026c 100644 --- a/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java +++ b/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java @@ -36,6 +36,11 @@ public CompletionStage> scheduleBatchLoader(ScheduledBatchLoaderC public CompletionStage> scheduleMappedBatchLoader(ScheduledMappedBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { return scheduledCall.invoke(); } + + @Override + public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + scheduledCall.invoke(); + } }; private BatchLoaderScheduler delayedScheduling(int ms) { @@ -56,6 +61,12 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp return scheduledCall.invoke(); }).thenCompose(Function.identity()); } + + @Override + public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + snooze(ms); + scheduledCall.invoke(); + } }; } @@ -139,6 +150,15 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp return scheduledCall.invoke(); }).thenCompose(Function.identity()); } + + @Override + public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + CompletableFuture.supplyAsync(() -> { + snooze(10); + scheduledCall.invoke(); + return null; + }); + } }; DataLoaderOptions options = DataLoaderOptions.newOptions().setBatchLoaderScheduler(funkyScheduler); From 6b5a732194b63a80f46dc4dc87214ac25fb860e0 Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sat, 18 May 2024 22:32:23 +1000 Subject: [PATCH 2/9] Eliminate *BatchObserver in favour of Publisher `reactive-streams` has become the de-facto standard for reactive frameworks; we thus use this as a base to allow seamless interop (rather than prompt an extra adapter layer). --- build.gradle | 7 +- .../java/org/dataloader/BatchObserver.java | 33 -------- .../java/org/dataloader/DataLoaderHelper.java | 75 +++++++++++-------- .../org/dataloader/MappedBatchObserver.java | 34 --------- .../MappedObserverBatchLoaderWithContext.java | 10 --- ...r.java => MappedPublisherBatchLoader.java} | 9 ++- ...MappedPublisherBatchLoaderWithContext.java | 13 ++++ .../ObserverBatchLoaderWithContext.java | 10 --- ...hLoader.java => PublisherBatchLoader.java} | 8 +- .../PublisherBatchLoaderWithContext.java | 12 +++ .../scheduler/BatchLoaderScheduler.java | 14 ++-- ...LoaderMappedPublisherBatchLoaderTest.java} | 38 ++-------- ...> DataLoaderPublisherBatchLoaderTest.java} | 33 +------- 13 files changed, 105 insertions(+), 191 deletions(-) delete mode 100644 src/main/java/org/dataloader/BatchObserver.java delete mode 100644 src/main/java/org/dataloader/MappedBatchObserver.java delete mode 100644 src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java rename src/main/java/org/dataloader/{MappedObserverBatchLoader.java => MappedPublisherBatchLoader.java} (63%) create mode 100644 src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java delete mode 100644 src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java rename src/main/java/org/dataloader/{ObserverBatchLoader.java => PublisherBatchLoader.java} (70%) create mode 100644 src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java rename src/test/java/org/dataloader/{DataLoaderObserverBatchLoaderTest.java => DataLoaderMappedPublisherBatchLoaderTest.java} (67%) rename src/test/java/org/dataloader/{DataLoaderMappedObserverBatchLoaderTest.java => DataLoaderPublisherBatchLoaderTest.java} (66%) diff --git a/build.gradle b/build.gradle index 6222df4..a22fc17 100644 --- a/build.gradle +++ b/build.gradle @@ -30,7 +30,6 @@ def getDevelopmentVersion() { version } -def slf4jVersion = '1.7.30' def releaseVersion = System.env.RELEASE_VERSION version = releaseVersion ? releaseVersion : getDevelopmentVersion() group = 'com.graphql-java' @@ -64,12 +63,18 @@ jar { } } +def slf4jVersion = '1.7.30' +def reactiveStreamsVersion = '1.0.3' + dependencies { api 'org.slf4j:slf4j-api:' + slf4jVersion + api 'org.reactivestreams:reactive-streams:' + reactiveStreamsVersion + testImplementation 'org.slf4j:slf4j-simple:' + slf4jVersion testImplementation 'junit:junit:4.12' testImplementation 'org.awaitility:awaitility:2.0.0' testImplementation 'com.github.ben-manes.caffeine:caffeine:2.9.0' + testImplementation 'io.projectreactor:reactor-core:3.6.6' } task sourcesJar(type: Jar) { diff --git a/src/main/java/org/dataloader/BatchObserver.java b/src/main/java/org/dataloader/BatchObserver.java deleted file mode 100644 index 14ef051..0000000 --- a/src/main/java/org/dataloader/BatchObserver.java +++ /dev/null @@ -1,33 +0,0 @@ -package org.dataloader; - -/** - * A interface intended as a delegate for other Observer-like classes used in other libraries, to be invoked by the calling - * {@link ObserverBatchLoader}. - *

- * Some examples include: - *

- * @param the value type of the {@link ObserverBatchLoader} - */ -public interface BatchObserver { - - /** - * To be called by the {@link ObserverBatchLoader} to load a new value. - */ - void onNext(V value); - - /** - * To be called by the {@link ObserverBatchLoader} to indicate all values have been successfully processed. - * This {@link BatchObserver} should not have any method invoked after this is called. - */ - void onCompleted(); - - /** - * To be called by the {@link ObserverBatchLoader} to indicate an unrecoverable error has been encountered. - * This {@link BatchObserver} should not have any method invoked after this is called. - */ - void onError(Throwable e); -} diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index 47d2d35..a7e1052 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -10,6 +10,8 @@ import org.dataloader.stats.context.IncrementCacheHitCountStatisticsContext; import org.dataloader.stats.context.IncrementLoadCountStatisticsContext; import org.dataloader.stats.context.IncrementLoadErrorCountStatisticsContext; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; import java.time.Clock; import java.time.Instant; @@ -246,7 +248,7 @@ private CompletableFuture> dispatchQueueBatch(List keys, List return batchLoad .thenApply(values -> { assertResultSize(keys, values); - if (isObserverLoader() || isMapObserverLoader()) { + if (isPublisherLoader() || isMappedPublisherLoader()) { // We have already completed the queued futures by the time the overall batchLoad future has completed. return values; } @@ -428,10 +430,10 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, .context(context).keyContexts(keys, keyContexts).build(); if (isMapLoader()) { batchLoad = invokeMapBatchLoader(keys, environment); - } else if (isObserverLoader()) { - batchLoad = invokeObserverBatchLoader(keys, keyContexts, queuedFutures, environment); - } else if (isMapObserverLoader()) { - batchLoad = invokeMappedObserverBatchLoader(keys, keyContexts, queuedFutures, environment); + } else if (isPublisherLoader()) { + batchLoad = invokePublisherBatchLoader(keys, keyContexts, queuedFutures, environment); + } else if (isMappedPublisherLoader()) { + batchLoad = invokeMappedPublisherBatchLoader(keys, keyContexts, queuedFutures, environment); } else { batchLoad = invokeListBatchLoader(keys, environment); } @@ -503,38 +505,38 @@ private CompletableFuture> invokeMapBatchLoader(List keys, BatchLoade }); } - private CompletableFuture> invokeObserverBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + private CompletableFuture> invokePublisherBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { CompletableFuture> loadResult = new CompletableFuture<>(); - BatchObserver observer = new BatchObserverImpl(loadResult, keys, keyContexts, queuedFutures); + Subscriber subscriber = new DataLoaderSubscriber(loadResult, keys, keyContexts, queuedFutures); BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); - if (batchLoadFunction instanceof ObserverBatchLoaderWithContext) { - ObserverBatchLoaderWithContext loadFunction = (ObserverBatchLoaderWithContext) batchLoadFunction; + if (batchLoadFunction instanceof PublisherBatchLoaderWithContext) { + PublisherBatchLoaderWithContext loadFunction = (PublisherBatchLoaderWithContext) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer, environment); + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, subscriber, environment); batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); } else { - loadFunction.load(keys, observer, environment); + loadFunction.load(keys, subscriber, environment); } } else { - ObserverBatchLoader loadFunction = (ObserverBatchLoader) batchLoadFunction; + PublisherBatchLoader loadFunction = (PublisherBatchLoader) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer); + BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, subscriber); batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); } else { - loadFunction.load(keys, observer); + loadFunction.load(keys, subscriber); } } return loadResult; } - private CompletableFuture> invokeMappedObserverBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + private CompletableFuture> invokeMappedPublisherBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { CompletableFuture> loadResult = new CompletableFuture<>(); - MappedBatchObserver observer = new MappedBatchObserverImpl(loadResult, keys, keyContexts, queuedFutures); + Subscriber> observer = new DataLoaderMapEntrySubscriber(loadResult, keys, keyContexts, queuedFutures); BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); - if (batchLoadFunction instanceof MappedObserverBatchLoaderWithContext) { - MappedObserverBatchLoaderWithContext loadFunction = (MappedObserverBatchLoaderWithContext) batchLoadFunction; + if (batchLoadFunction instanceof MappedPublisherBatchLoaderWithContext) { + MappedPublisherBatchLoaderWithContext loadFunction = (MappedPublisherBatchLoaderWithContext) batchLoadFunction; if (batchLoaderScheduler != null) { BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer, environment); batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); @@ -542,7 +544,7 @@ private CompletableFuture> invokeMappedObserverBatchLoader(List keys, loadFunction.load(keys, observer, environment); } } else { - MappedObserverBatchLoader loadFunction = (MappedObserverBatchLoader) batchLoadFunction; + MappedPublisherBatchLoader loadFunction = (MappedPublisherBatchLoader) batchLoadFunction; if (batchLoaderScheduler != null) { BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer); batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); @@ -557,12 +559,12 @@ private boolean isMapLoader() { return batchLoadFunction instanceof MappedBatchLoader || batchLoadFunction instanceof MappedBatchLoaderWithContext; } - private boolean isObserverLoader() { - return batchLoadFunction instanceof ObserverBatchLoader; + private boolean isPublisherLoader() { + return batchLoadFunction instanceof PublisherBatchLoader; } - private boolean isMapObserverLoader() { - return batchLoadFunction instanceof MappedObserverBatchLoader; + private boolean isMappedPublisherLoader() { + return batchLoadFunction instanceof MappedPublisherBatchLoader; } int dispatchDepth() { @@ -616,7 +618,8 @@ private static DispatchResult emptyDispatchResult() { return (DispatchResult) EMPTY_DISPATCH_RESULT; } - private class BatchObserverImpl implements BatchObserver { + private class DataLoaderSubscriber implements Subscriber { + private final CompletableFuture> valuesFuture; private final List keys; private final List callContexts; @@ -628,7 +631,7 @@ private class BatchObserverImpl implements BatchObserver { private boolean onErrorCalled = false; private boolean onCompletedCalled = false; - private BatchObserverImpl( + private DataLoaderSubscriber( CompletableFuture> valuesFuture, List keys, List callContexts, @@ -640,6 +643,11 @@ private BatchObserverImpl( this.queuedFutures = queuedFutures; } + @Override + public void onSubscribe(Subscription subscription) { + subscription.request(keys.size()); + } + @Override public void onNext(V value) { assert !onErrorCalled && !onCompletedCalled; @@ -671,7 +679,7 @@ public void onNext(V value) { } @Override - public void onCompleted() { + public void onComplete() { assert !onErrorCalled; onCompletedCalled = true; @@ -701,7 +709,7 @@ public void onError(Throwable ex) { } } - private class MappedBatchObserverImpl implements MappedBatchObserver { + private class DataLoaderMapEntrySubscriber implements Subscriber> { private final CompletableFuture> valuesFuture; private final List keys; private final List callContexts; @@ -714,7 +722,7 @@ private class MappedBatchObserverImpl implements MappedBatchObserver { private boolean onErrorCalled = false; private boolean onCompletedCalled = false; - private MappedBatchObserverImpl( + private DataLoaderMapEntrySubscriber( CompletableFuture> valuesFuture, List keys, List callContexts, @@ -737,8 +745,15 @@ private MappedBatchObserverImpl( } @Override - public void onNext(K key, V value) { + public void onSubscribe(Subscription subscription) { + subscription.request(keys.size()); + } + + @Override + public void onNext(Map.Entry entry) { assert !onErrorCalled && !onCompletedCalled; + K key = entry.getKey(); + V value = entry.getValue(); Object callContext = callContextByKey.get(key); CompletableFuture future = queuedFutureByKey.get(key); @@ -765,7 +780,7 @@ public void onNext(K key, V value) { } @Override - public void onCompleted() { + public void onComplete() { assert !onErrorCalled; onCompletedCalled = true; diff --git a/src/main/java/org/dataloader/MappedBatchObserver.java b/src/main/java/org/dataloader/MappedBatchObserver.java deleted file mode 100644 index 59a0f73..0000000 --- a/src/main/java/org/dataloader/MappedBatchObserver.java +++ /dev/null @@ -1,34 +0,0 @@ -package org.dataloader; - -/** - * A interface intended as a delegate for other Observer-like classes used in other libraries, to be invoked by the calling - * {@link MappedObserverBatchLoader}. - *

- * Some examples include: - *

- * @param the key type of the calling {@link MappedObserverBatchLoader}. - * @param the value type of the calling {@link MappedObserverBatchLoader}. - */ -public interface MappedBatchObserver { - - /** - * To be called by the {@link MappedObserverBatchLoader} to process a new key/value pair. - */ - void onNext(K key, V value); - - /** - * To be called by the {@link MappedObserverBatchLoader} to indicate all values have been successfully processed. - * This {@link MappedBatchObserver} should not have any method invoked after this method is called. - */ - void onCompleted(); - - /** - * To be called by the {@link MappedObserverBatchLoader} to indicate an unrecoverable error has been encountered. - * This {@link MappedBatchObserver} should not have any method invoked after this method is called. - */ - void onError(Throwable e); -} diff --git a/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java b/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java deleted file mode 100644 index 6619198..0000000 --- a/src/main/java/org/dataloader/MappedObserverBatchLoaderWithContext.java +++ /dev/null @@ -1,10 +0,0 @@ -package org.dataloader; - -import java.util.List; - -/** - * A {@link MappedObserverBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. - */ -public interface MappedObserverBatchLoaderWithContext { - void load(List keys, MappedBatchObserver observer, BatchLoaderEnvironment environment); -} diff --git a/src/main/java/org/dataloader/MappedObserverBatchLoader.java b/src/main/java/org/dataloader/MappedPublisherBatchLoader.java similarity index 63% rename from src/main/java/org/dataloader/MappedObserverBatchLoader.java rename to src/main/java/org/dataloader/MappedPublisherBatchLoader.java index d82ec75..9c7430a 100644 --- a/src/main/java/org/dataloader/MappedObserverBatchLoader.java +++ b/src/main/java/org/dataloader/MappedPublisherBatchLoader.java @@ -1,17 +1,20 @@ package org.dataloader; +import org.reactivestreams.Subscriber; + import java.util.List; +import java.util.Map; /** * A function that is invoked for batch loading a stream of data values indicated by the provided list of keys. *

- * The function will call the provided {@link MappedBatchObserver} to process the key/value pairs it has retrieved to allow + * The function will call the provided {@link Subscriber} to process the key/value pairs it has retrieved to allow * the future returned by {@link DataLoader#load(Object)} to complete as soon as the individual value is available * (rather than when all values have been retrieved). * * @param type parameter indicating the type of keys to use for data load requests. * @param type parameter indicating the type of values returned */ -public interface MappedObserverBatchLoader { - void load(List keys, MappedBatchObserver observer); +public interface MappedPublisherBatchLoader { + void load(List keys, Subscriber> subscriber); } diff --git a/src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java b/src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java new file mode 100644 index 0000000..a752abc --- /dev/null +++ b/src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java @@ -0,0 +1,13 @@ +package org.dataloader; + +import org.reactivestreams.Subscriber; + +import java.util.List; +import java.util.Map; + +/** + * A {@link MappedPublisherBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + */ +public interface MappedPublisherBatchLoaderWithContext { + void load(List keys, Subscriber> subscriber, BatchLoaderEnvironment environment); +} diff --git a/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java b/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java deleted file mode 100644 index 14a3dd1..0000000 --- a/src/main/java/org/dataloader/ObserverBatchLoaderWithContext.java +++ /dev/null @@ -1,10 +0,0 @@ -package org.dataloader; - -import java.util.List; - -/** - * An {@link ObserverBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. - */ -public interface ObserverBatchLoaderWithContext { - void load(List keys, BatchObserver observer, BatchLoaderEnvironment environment); -} diff --git a/src/main/java/org/dataloader/ObserverBatchLoader.java b/src/main/java/org/dataloader/PublisherBatchLoader.java similarity index 70% rename from src/main/java/org/dataloader/ObserverBatchLoader.java rename to src/main/java/org/dataloader/PublisherBatchLoader.java index 0c481f9..2dcdf1e 100644 --- a/src/main/java/org/dataloader/ObserverBatchLoader.java +++ b/src/main/java/org/dataloader/PublisherBatchLoader.java @@ -1,11 +1,13 @@ package org.dataloader; +import org.reactivestreams.Subscriber; + import java.util.List; /** * A function that is invoked for batch loading a stream of data values indicated by the provided list of keys. *

- * The function will call the provided {@link BatchObserver} to process the values it has retrieved to allow + * The function will call the provided {@link Subscriber} to process the values it has retrieved to allow * the future returned by {@link DataLoader#load(Object)} to complete as soon as the individual value is available * (rather than when all values have been retrieved). *

@@ -14,6 +16,6 @@ * @param type parameter indicating the type of keys to use for data load requests. * @param type parameter indicating the type of values returned */ -public interface ObserverBatchLoader { - void load(List keys, BatchObserver observer); +public interface PublisherBatchLoader { + void load(List keys, Subscriber subscriber); } diff --git a/src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java b/src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java new file mode 100644 index 0000000..45ea36d --- /dev/null +++ b/src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java @@ -0,0 +1,12 @@ +package org.dataloader; + +import org.reactivestreams.Subscriber; + +import java.util.List; + +/** + * An {@link PublisherBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + */ +public interface PublisherBatchLoaderWithContext { + void load(List keys, Subscriber subscriber, BatchLoaderEnvironment environment); +} diff --git a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java index 5b88d2c..2e82eff 100644 --- a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java +++ b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java @@ -5,8 +5,8 @@ import org.dataloader.DataLoader; import org.dataloader.DataLoaderOptions; import org.dataloader.MappedBatchLoader; -import org.dataloader.MappedObserverBatchLoader; -import org.dataloader.ObserverBatchLoader; +import org.dataloader.MappedPublisherBatchLoader; +import org.dataloader.PublisherBatchLoader; import java.util.List; import java.util.Map; @@ -45,7 +45,7 @@ interface ScheduledMappedBatchLoaderCall { } /** - * This represents a callback that will invoke a {@link ObserverBatchLoader} or {@link MappedObserverBatchLoader} function under the covers + * This represents a callback that will invoke a {@link PublisherBatchLoader} or {@link MappedPublisherBatchLoader} function under the covers */ interface ScheduledObserverBatchLoaderCall { void invoke(); @@ -82,13 +82,13 @@ interface ScheduledObserverBatchLoaderCall { CompletionStage> scheduleMappedBatchLoader(ScheduledMappedBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); /** - * This is called to schedule a {@link ObserverBatchLoader} call. + * This is called to schedule a {@link PublisherBatchLoader} call. * - * @param scheduledCall the callback that needs to be invoked to allow the {@link ObserverBatchLoader} to proceed. - * @param keys this is the list of keys that will be passed to the {@link ObserverBatchLoader}. + * @param scheduledCall the callback that needs to be invoked to allow the {@link PublisherBatchLoader} to proceed. + * @param keys this is the list of keys that will be passed to the {@link PublisherBatchLoader}. * This is provided only for informative reasons and, you can't change the keys that are used * @param environment this is the {@link BatchLoaderEnvironment} in place, - * which can be null if it's a simple {@link ObserverBatchLoader} call + * which can be null if it's a simple {@link PublisherBatchLoader} call * @param the key type */ void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); diff --git a/src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java similarity index 67% rename from src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java rename to src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java index eaeef8f..82d6c29 100644 --- a/src/test/java/org/dataloader/DataLoaderObserverBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java @@ -1,8 +1,10 @@ package org.dataloader; import org.junit.Test; +import reactor.core.publisher.Flux; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; @@ -15,7 +17,7 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -public class DataLoaderObserverBatchLoaderTest { +public class DataLoaderMappedPublisherBatchLoaderTest { @Test public void should_Build_a_really_really_simple_data_loader() { @@ -74,35 +76,9 @@ public void should_observer_batch_multiple_requests() throws ExecutionException, assertThat(future2.get(), equalTo(2)); } - // A simple wrapper class intended as a proof external libraries can leverage this. - private static class Publisher { - private final BatchObserver delegate; - private Publisher(BatchObserver delegate) { this.delegate = delegate; } - void onNext(V value) { delegate.onNext(value); } - void onCompleted() { delegate.onCompleted(); } - void onError(Throwable e) { delegate.onError(e); } - // Mock 'subscribe' methods to simulate what would happen in the real thing. - void subscribe(List values) { - values.forEach(this::onNext); - this.onCompleted(); - } - void subscribe(List values, Throwable e) { - values.forEach(this::onNext); - this.onError(e); - } - } - - private static ObserverBatchLoader keysAsValues() { - return (keys, observer) -> { - Publisher publisher = new Publisher<>(observer); - publisher.subscribe(keys); - }; - } - - private static ObserverBatchLoader keysWithValuesAndException(List values, Throwable e) { - return (keys, observer) -> { - Publisher publisher = new Publisher<>(observer); - publisher.subscribe(values, e); - }; + private static MappedPublisherBatchLoader keysAsValues() { + return (keys, subscriber) -> Flux + .fromStream(keys.stream().map(k -> Map.entry(k, k))) + .subscribe(subscriber); } } diff --git a/src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java similarity index 66% rename from src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java rename to src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java index e6f1168..a286ac8 100644 --- a/src/test/java/org/dataloader/DataLoaderMappedObserverBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java @@ -1,24 +1,22 @@ package org.dataloader; import org.junit.Test; +import reactor.core.publisher.Flux; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Arrays.asList; -import static java.util.function.Function.identity; -import static java.util.stream.Collectors.toMap; import static org.awaitility.Awaitility.await; import static org.dataloader.DataLoaderFactory.mkDataLoader; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -public class DataLoaderMappedObserverBatchLoaderTest { +public class DataLoaderPublisherBatchLoaderTest { @Test public void should_Build_a_really_really_simple_data_loader() { @@ -77,30 +75,7 @@ public void should_observer_batch_multiple_requests() throws ExecutionException, assertThat(future2.get(), equalTo(2)); } - // A simple wrapper class intended as a proof external libraries can leverage this. - private static class Publisher { - - private final MappedBatchObserver delegate; - private Publisher(MappedBatchObserver delegate) { this.delegate = delegate; } - void onNext(Map.Entry entry) { delegate.onNext(entry.getKey(), entry.getValue()); } - void onCompleted() { delegate.onCompleted(); } - void onError(Throwable e) { delegate.onError(e); } - // Mock 'subscribe' methods to simulate what would happen in the real thing. - void subscribe(Map valueByKey) { - valueByKey.entrySet().forEach(this::onNext); - this.onCompleted(); - } - void subscribe(Map valueByKey, Throwable e) { - valueByKey.entrySet().forEach(this::onNext); - this.onError(e); - } - } - - private static MappedObserverBatchLoader keysAsValues() { - return (keys, observer) -> { - Publisher publisher = new Publisher<>(observer); - Map valueByKey = keys.stream().collect(toMap(identity(), identity())); - publisher.subscribe(valueByKey); - }; + private static PublisherBatchLoader keysAsValues() { + return (keys, subscriber) -> Flux.fromIterable(keys).subscribe(subscriber); } } From 68d7f54984fe567f20e71f9d64da428a137323d7 Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sat, 18 May 2024 22:44:45 +1000 Subject: [PATCH 3/9] Use internal Assertions over Java's raw assert This gives us more workable exceptions. --- .../java/org/dataloader/DataLoaderHelper.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index a7e1052..df13e0e 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -629,7 +629,7 @@ private class DataLoaderSubscriber implements Subscriber { private final List completedValues = new ArrayList<>(); private int idx = 0; private boolean onErrorCalled = false; - private boolean onCompletedCalled = false; + private boolean onCompleteCalled = false; private DataLoaderSubscriber( CompletableFuture> valuesFuture, @@ -650,7 +650,8 @@ public void onSubscribe(Subscription subscription) { @Override public void onNext(V value) { - assert !onErrorCalled && !onCompletedCalled; + assertState(!onErrorCalled, () -> "onError has already been called; onNext may not be invoked."); + assertState(!onCompleteCalled, () -> "onComplete has already been called; onNext may not be invoked."); K key = keys.get(idx); Object callContext = callContexts.get(idx); @@ -680,8 +681,8 @@ public void onNext(V value) { @Override public void onComplete() { - assert !onErrorCalled; - onCompletedCalled = true; + assertState(!onErrorCalled, () -> "onError has already been called; onComplete may not be invoked."); + onCompleteCalled = true; assertResultSize(keys, completedValues); @@ -691,7 +692,7 @@ public void onComplete() { @Override public void onError(Throwable ex) { - assert !onCompletedCalled; + assertState(!onCompleteCalled, () -> "onComplete has already been called; onError may not be invoked."); onErrorCalled = true; stats.incrementBatchLoadExceptionCount(new IncrementBatchLoadExceptionCountStatisticsContext<>(keys, callContexts)); @@ -720,7 +721,7 @@ private class DataLoaderMapEntrySubscriber implements Subscriber private final List clearCacheKeys = new ArrayList<>(); private final Map completedValuesByKey = new HashMap<>(); private boolean onErrorCalled = false; - private boolean onCompletedCalled = false; + private boolean onCompleteCalled = false; private DataLoaderMapEntrySubscriber( CompletableFuture> valuesFuture, @@ -751,7 +752,8 @@ public void onSubscribe(Subscription subscription) { @Override public void onNext(Map.Entry entry) { - assert !onErrorCalled && !onCompletedCalled; + assertState(!onErrorCalled, () -> "onError has already been called; onNext may not be invoked."); + assertState(!onCompleteCalled, () -> "onComplete has already been called; onNext may not be invoked."); K key = entry.getKey(); V value = entry.getValue(); @@ -781,8 +783,8 @@ public void onNext(Map.Entry entry) { @Override public void onComplete() { - assert !onErrorCalled; - onCompletedCalled = true; + assertState(!onErrorCalled, () -> "onError has already been called; onComplete may not be invoked."); + onCompleteCalled = true; possiblyClearCacheEntriesOnExceptions(clearCacheKeys); List values = new ArrayList<>(keys.size()); @@ -795,7 +797,7 @@ public void onComplete() { @Override public void onError(Throwable ex) { - assert !onCompletedCalled; + assertState(!onCompleteCalled, () -> "onComplete has already been called; onError may not be invoked."); onErrorCalled = true; stats.incrementBatchLoadExceptionCount(new IncrementBatchLoadExceptionCountStatisticsContext<>(keys, callContexts)); From a3132b71631fe7aeaa358825fee518a05dfea357 Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sat, 18 May 2024 22:52:34 +1000 Subject: [PATCH 4/9] Remove handling of Throwable passed into onNext Passing an exception into `onNext` is not typically done in reactive-land - we would instead call `onError(Throwable)`. We can thus avoid handling this case. --- src/main/java/org/dataloader/DataLoaderHelper.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index df13e0e..12817e1 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -656,11 +656,7 @@ public void onNext(V value) { K key = keys.get(idx); Object callContext = callContexts.get(idx); CompletableFuture future = queuedFutures.get(idx); - if (value instanceof Throwable) { - stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); - future.completeExceptionally((Throwable) value); - clearCacheKeys.add(keys.get(idx)); - } else if (value instanceof Try) { + if (value instanceof Try) { // we allow the batch loader to return a Try so we can better represent a computation // that might have worked or not. Try tryValue = (Try) value; @@ -759,11 +755,7 @@ public void onNext(Map.Entry entry) { Object callContext = callContextByKey.get(key); CompletableFuture future = queuedFutureByKey.get(key); - if (value instanceof Throwable) { - stats.incrementLoadErrorCount(new IncrementLoadErrorCountStatisticsContext<>(key, callContext)); - future.completeExceptionally((Throwable) value); - clearCacheKeys.add(key); - } else if (value instanceof Try) { + if (value instanceof Try) { // we allow the batch loader to return a Try so we can better represent a computation // that might have worked or not. Try tryValue = (Try) value; From fbeffae774d965e35af7b1b83be88d7e629171ed Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sat, 18 May 2024 23:24:40 +1000 Subject: [PATCH 5/9] Expose `new*DataLoader` methods for *PublisherBatchLoader This is keeping in line with the other methods found in `DataLoaderFactory`. --- .../org/dataloader/DataLoaderFactory.java | 268 ++++++++++++++++++ ...aLoaderMappedPublisherBatchLoaderTest.java | 10 +- .../DataLoaderPublisherBatchLoaderTest.java | 10 +- 3 files changed, 278 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/dataloader/DataLoaderFactory.java b/src/main/java/org/dataloader/DataLoaderFactory.java index 013f473..5b50874 100644 --- a/src/main/java/org/dataloader/DataLoaderFactory.java +++ b/src/main/java/org/dataloader/DataLoaderFactory.java @@ -278,6 +278,274 @@ public static DataLoader newMappedDataLoaderWithTry(MappedBatchLoad return mkDataLoader(batchLoadFunction, options); } + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size). + * + * @param batchLoadFunction the batch load function to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoader(PublisherBatchLoader batchLoadFunction) { + return newPublisherDataLoader(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function with the provided options + * + * @param batchLoadFunction the batch load function to use + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoader(PublisherBatchLoader batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size) where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + *

+ * If it's important you to know the exact status of each item in a batch call and whether it threw exceptions then + * you can use this form to create the data loader. + *

+ * Using Try objects allows you to capture a value returned or an exception that might + * have occurred trying to get a value. . + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoader> batchLoadFunction) { + return newPublisherDataLoaderWithTry(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function and with the provided options + * where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + * + * @see #newDataLoaderWithTry(BatchLoader) + */ + public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoader> batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size). + * + * @param batchLoadFunction the batch load function to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoader(PublisherBatchLoaderWithContext batchLoadFunction) { + return newPublisherDataLoader(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function with the provided options + * + * @param batchLoadFunction the batch load function to use + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoader(PublisherBatchLoaderWithContext batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size) where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + *

+ * If it's important you to know the exact status of each item in a batch call and whether it threw exceptions then + * you can use this form to create the data loader. + *

+ * Using Try objects allows you to capture a value returned or an exception that might + * have occurred trying to get a value. . + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoaderWithContext> batchLoadFunction) { + return newPublisherDataLoaderWithTry(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function and with the provided options + * where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + * + * @see #newPublisherDataLoaderWithTry(PublisherBatchLoader) + */ + public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoaderWithContext> batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size). + * + * @param batchLoadFunction the batch load function to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoader batchLoadFunction) { + return newMappedPublisherDataLoader(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function with the provided options + * + * @param batchLoadFunction the batch load function to use + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoader batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size) where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + *

+ * If it's important you to know the exact status of each item in a batch call and whether it threw exceptions then + * you can use this form to create the data loader. + *

+ * Using Try objects allows you to capture a value returned or an exception that might + * have occurred trying to get a value. . + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader> batchLoadFunction) { + return newMappedPublisherDataLoaderWithTry(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function and with the provided options + * where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + * + * @see #newDataLoaderWithTry(BatchLoader) + */ + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader> batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size). + * + * @param batchLoadFunction the batch load function to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoaderWithContext batchLoadFunction) { + return newMappedPublisherDataLoader(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function with the provided options + * + * @param batchLoadFunction the batch load function to use + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoaderWithContext batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + + /** + * Creates new DataLoader with the specified batch loader function and default options + * (batching, caching and unlimited batch size) where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + *

+ * If it's important you to know the exact status of each item in a batch call and whether it threw exceptions then + * you can use this form to create the data loader. + *

+ * Using Try objects allows you to capture a value returned or an exception that might + * have occurred trying to get a value. . + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param the key type + * @param the value type + * + * @return a new DataLoader + */ + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoaderWithContext> batchLoadFunction) { + return newMappedPublisherDataLoaderWithTry(batchLoadFunction, null); + } + + /** + * Creates new DataLoader with the specified batch loader function and with the provided options + * where the batch loader function returns a list of + * {@link org.dataloader.Try} objects. + * + * @param batchLoadFunction the batch load function to use that uses {@link org.dataloader.Try} objects + * @param options the options to use + * @param the key type + * @param the value type + * + * @return a new DataLoader + * + * @see #newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader) + */ + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoaderWithContext> batchLoadFunction, DataLoaderOptions options) { + return mkDataLoader(batchLoadFunction, options); + } + static DataLoader mkDataLoader(Object batchLoadFunction, DataLoaderOptions options) { return new DataLoader<>(batchLoadFunction, options); } diff --git a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java index 82d6c29..757abb7 100644 --- a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java @@ -12,7 +12,7 @@ import static java.util.Arrays.asList; import static org.awaitility.Awaitility.await; -import static org.dataloader.DataLoaderFactory.mkDataLoader; +import static org.dataloader.DataLoaderFactory.newMappedPublisherDataLoader; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -22,7 +22,7 @@ public class DataLoaderMappedPublisherBatchLoaderTest { @Test public void should_Build_a_really_really_simple_data_loader() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); CompletionStage future1 = identityLoader.load(1); @@ -37,7 +37,7 @@ public void should_Build_a_really_really_simple_data_loader() { @Test public void should_Support_loading_multiple_keys_in_one_call() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); futureAll.thenAccept(promisedValues -> { @@ -51,7 +51,7 @@ public void should_Support_loading_multiple_keys_in_one_call() { @Test public void simple_dataloader() { - DataLoader loader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader loader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); loader.load("A"); loader.load("B"); @@ -65,7 +65,7 @@ public void simple_dataloader() { @Test public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { - DataLoader identityLoader = mkDataLoader(keysAsValues(), new DataLoaderOptions()); + DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), new DataLoaderOptions()); CompletableFuture future1 = identityLoader.load(1); CompletableFuture future2 = identityLoader.load(2); diff --git a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java index a286ac8..4e5d3e1 100644 --- a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java @@ -11,7 +11,7 @@ import static java.util.Arrays.asList; import static org.awaitility.Awaitility.await; -import static org.dataloader.DataLoaderFactory.mkDataLoader; +import static org.dataloader.DataLoaderFactory.newPublisherDataLoader; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -21,7 +21,7 @@ public class DataLoaderPublisherBatchLoaderTest { @Test public void should_Build_a_really_really_simple_data_loader() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); CompletionStage future1 = identityLoader.load(1); @@ -36,7 +36,7 @@ public void should_Build_a_really_really_simple_data_loader() { @Test public void should_Support_loading_multiple_keys_in_one_call() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); futureAll.thenAccept(promisedValues -> { @@ -50,7 +50,7 @@ public void should_Support_loading_multiple_keys_in_one_call() { @Test public void simple_dataloader() { - DataLoader loader = mkDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader loader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); loader.load("A"); loader.load("B"); @@ -64,7 +64,7 @@ public void simple_dataloader() { @Test public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { - DataLoader identityLoader = mkDataLoader(keysAsValues(), new DataLoaderOptions()); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), new DataLoaderOptions()); CompletableFuture future1 = identityLoader.load(1); CompletableFuture future2 = identityLoader.load(2); From b2a662da67dcd89285f8fae83c11ac90f38805bb Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sun, 19 May 2024 00:33:35 +1000 Subject: [PATCH 6/9] Copy/tweak original/ DataLoader tests for publisher equivalents Given the large number of existing tests, we copy across this existing set for our publisher tests. What this really indicates is that we should invest in parameterised testing, but this is a bit painful in JUnit 4 - so we'll bump to JUnit 5 independently and parameterise when we have this available. This is important because re-using the existing test suite reveals a failure that we'll need to address. --- ...aLoaderMappedPublisherBatchLoaderTest.java | 161 ++- .../DataLoaderPublisherBatchLoaderTest.java | 1041 ++++++++++++++++- 2 files changed, 1154 insertions(+), 48 deletions(-) diff --git a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java index 757abb7..8e33300 100644 --- a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java @@ -3,55 +3,65 @@ import org.junit.Test; import reactor.core.publisher.Flux; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.awaitility.Awaitility.await; import static org.dataloader.DataLoaderFactory.newMappedPublisherDataLoader; +import static org.dataloader.DataLoaderOptions.newOptions; +import static org.dataloader.fixtures.TestKit.listFrom; +import static org.dataloader.impl.CompletableFutureKit.cause; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; public class DataLoaderMappedPublisherBatchLoaderTest { - @Test - public void should_Build_a_really_really_simple_data_loader() { - AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); - - CompletionStage future1 = identityLoader.load(1); + MappedPublisherBatchLoader evensOnlyMappedBatchLoader = (keys, subscriber) -> { + Map mapOfResults = new HashMap<>(); - future1.thenAccept(value -> { - assertThat(value, equalTo(1)); - success.set(true); + AtomicInteger index = new AtomicInteger(); + keys.forEach(k -> { + int i = index.getAndIncrement(); + if (i % 2 == 0) { + mapOfResults.put(k, k); + } }); - identityLoader.dispatch(); - await().untilAtomic(success, is(true)); + Flux.fromIterable(mapOfResults.entrySet()).subscribe(subscriber); + }; + + private static DataLoader idMapLoader(DataLoaderOptions options, List> loadCalls) { + MappedPublisherBatchLoader kvBatchLoader = (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + Map map = new HashMap<>(); + //noinspection unchecked + keys.forEach(k -> map.put(k, (V) k)); + Flux.fromIterable(map.entrySet()).subscribe(subscriber); + }; + return DataLoaderFactory.newMappedPublisherDataLoader(kvBatchLoader, options); } - @Test - public void should_Support_loading_multiple_keys_in_one_call() { - AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); - - CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); - futureAll.thenAccept(promisedValues -> { - assertThat(promisedValues.size(), is(2)); - success.set(true); - }); - identityLoader.dispatch(); - await().untilAtomic(success, is(true)); - assertThat(futureAll.toCompletableFuture().join(), equalTo(asList(1, 2))); + private static DataLoader idMapLoaderBlowsUps( + DataLoaderOptions options, List> loadCalls) { + return newMappedPublisherDataLoader((MappedPublisherBatchLoader) (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + Flux.>error(new IllegalStateException("Error")).subscribe(subscriber); + }, options); } + @Test - public void simple_dataloader() { - DataLoader loader = newMappedPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + public void basic_map_batch_loading() { + DataLoader loader = DataLoaderFactory.newMappedPublisherDataLoader(evensOnlyMappedBatchLoader); loader.load("A"); loader.load("B"); @@ -60,12 +70,13 @@ public void simple_dataloader() { List results = loader.dispatchAndJoin(); assertThat(results.size(), equalTo(4)); - assertThat(results, equalTo(asList("A", "B", "C", "D"))); + assertThat(results, equalTo(asList("A", null, "C", null))); } @Test - public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { - DataLoader identityLoader = newMappedPublisherDataLoader(keysAsValues(), new DataLoaderOptions()); + public void should_map_Batch_multiple_requests() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idMapLoader(new DataLoaderOptions(), loadCalls); CompletableFuture future1 = identityLoader.load(1); CompletableFuture future2 = identityLoader.load(2); @@ -74,11 +85,91 @@ public void should_observer_batch_multiple_requests() throws ExecutionException, await().until(() -> future1.isDone() && future2.isDone()); assertThat(future1.get(), equalTo(1)); assertThat(future2.get(), equalTo(2)); + assertThat(loadCalls, equalTo(singletonList(asList(1, 2)))); + } + + @Test + public void can_split_max_batch_sizes_correctly() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idMapLoader(newOptions().setMaxBatchSize(5), loadCalls); + + for (int i = 0; i < 21; i++) { + identityLoader.load(i); + } + List> expectedCalls = new ArrayList<>(); + expectedCalls.add(listFrom(0, 5)); + expectedCalls.add(listFrom(5, 10)); + expectedCalls.add(listFrom(10, 15)); + expectedCalls.add(listFrom(15, 20)); + expectedCalls.add(listFrom(20, 21)); + + List result = identityLoader.dispatch().join(); + + assertThat(result, equalTo(listFrom(0, 21))); + assertThat(loadCalls, equalTo(expectedCalls)); + } + + @Test + public void should_Propagate_error_to_all_loads() { + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idMapLoaderBlowsUps(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + CompletableFuture future2 = errorLoader.load(2); + errorLoader.dispatch(); + + await().until(future1::isDone); + + assertThat(future1.isCompletedExceptionally(), is(true)); + Throwable cause = cause(future1); + assert cause != null; + assertThat(cause, instanceOf(IllegalStateException.class)); + assertThat(cause.getMessage(), equalTo("Error")); + + await().until(future2::isDone); + cause = cause(future2); + assert cause != null; + assertThat(cause.getMessage(), equalTo(cause.getMessage())); + + assertThat(loadCalls, equalTo(singletonList(asList(1, 2)))); + } + + @Test + public void should_work_with_duplicate_keys_when_caching_disabled() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = + idMapLoader(newOptions().setCachingEnabled(false), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture future3 = identityLoader.load("A"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone() && future3.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(future3.get(), equalTo("A")); + + // the map batch functions use a set of keys as input and hence remove duplicates unlike list variant + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); } - private static MappedPublisherBatchLoader keysAsValues() { - return (keys, subscriber) -> Flux - .fromStream(keys.stream().map(k -> Map.entry(k, k))) - .subscribe(subscriber); + @Test + public void should_work_with_duplicate_keys_when_caching_enabled() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = + idMapLoader(newOptions().setCachingEnabled(true), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture future3 = identityLoader.load("A"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone() && future3.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(future3.get(), equalTo("A")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); } + } diff --git a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java index 4e5d3e1..508a031 100644 --- a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java @@ -1,19 +1,40 @@ package org.dataloader; +import org.dataloader.fixtures.CustomCacheMap; +import org.dataloader.fixtures.JsonObject; +import org.dataloader.fixtures.User; +import org.dataloader.fixtures.UserManager; +import org.dataloader.impl.CompletableFutureKit; import org.junit.Test; import reactor.core.publisher.Flux; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; import static org.awaitility.Awaitility.await; +import static org.dataloader.DataLoaderFactory.newDataLoader; import static org.dataloader.DataLoaderFactory.newPublisherDataLoader; +import static org.dataloader.DataLoaderFactory.newPublisherDataLoaderWithTry; +import static org.dataloader.DataLoaderOptions.newOptions; +import static org.dataloader.fixtures.TestKit.listFrom; +import static org.dataloader.impl.CompletableFutureKit.cause; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThat; public class DataLoaderPublisherBatchLoaderTest { @@ -21,7 +42,7 @@ public class DataLoaderPublisherBatchLoaderTest { @Test public void should_Build_a_really_really_simple_data_loader() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues()); CompletionStage future1 = identityLoader.load(1); @@ -36,7 +57,7 @@ public void should_Build_a_really_really_simple_data_loader() { @Test public void should_Support_loading_multiple_keys_in_one_call() { AtomicBoolean success = new AtomicBoolean(); - DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues()); CompletionStage> futureAll = identityLoader.loadMany(asList(1, 2)); futureAll.thenAccept(promisedValues -> { @@ -49,33 +70,1027 @@ public void should_Support_loading_multiple_keys_in_one_call() { } @Test - public void simple_dataloader() { - DataLoader loader = newPublisherDataLoader(keysAsValues(), DataLoaderOptions.newOptions()); + public void should_Resolve_to_empty_list_when_no_keys_supplied() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues()); + CompletableFuture> futureEmpty = identityLoader.loadMany(emptyList()); + futureEmpty.thenAccept(promisedValues -> { + assertThat(promisedValues.size(), is(0)); + success.set(true); + }); + identityLoader.dispatch(); + await().untilAtomic(success, is(true)); + assertThat(futureEmpty.join(), empty()); + } - loader.load("A"); - loader.load("B"); - loader.loadMany(asList("C", "D")); + @Test + public void should_Return_zero_entries_dispatched_when_no_keys_supplied() { + AtomicBoolean success = new AtomicBoolean(); + DataLoader identityLoader = newPublisherDataLoader(keysAsValues()); + CompletableFuture> futureEmpty = identityLoader.loadMany(emptyList()); + futureEmpty.thenAccept(promisedValues -> { + assertThat(promisedValues.size(), is(0)); + success.set(true); + }); + DispatchResult dispatchResult = identityLoader.dispatchWithCounts(); + await().untilAtomic(success, is(true)); + assertThat(dispatchResult.getKeysCount(), equalTo(0)); + } - List results = loader.dispatchAndJoin(); + @Test + public void should_Batch_multiple_requests() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load(1); + CompletableFuture future2 = identityLoader.load(2); + identityLoader.dispatch(); - assertThat(results.size(), equalTo(4)); - assertThat(results, equalTo(asList("A", "B", "C", "D"))); + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo(1)); + assertThat(future2.get(), equalTo(2)); + assertThat(loadCalls, equalTo(singletonList(asList(1, 2)))); } @Test - public void should_observer_batch_multiple_requests() throws ExecutionException, InterruptedException { - DataLoader identityLoader = newPublisherDataLoader(keysAsValues(), new DataLoaderOptions()); + public void should_Return_number_of_batched_entries() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); CompletableFuture future1 = identityLoader.load(1); CompletableFuture future2 = identityLoader.load(2); + DispatchResult dispatchResult = identityLoader.dispatchWithCounts(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(dispatchResult.getKeysCount(), equalTo(2)); // its two because it's the number dispatched (by key) not the load calls + assertThat(dispatchResult.getPromisedResults().isDone(), equalTo(true)); + } + + @Test + public void should_Coalesce_identical_requests() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1a = identityLoader.load(1); + CompletableFuture future1b = identityLoader.load(1); + assertThat(future1a, equalTo(future1b)); + identityLoader.dispatch(); + + await().until(future1a::isDone); + assertThat(future1a.get(), equalTo(1)); + assertThat(future1b.get(), equalTo(1)); + assertThat(loadCalls, equalTo(singletonList(singletonList(1)))); + } + + @Test + public void should_Cache_repeated_requests() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); identityLoader.dispatch(); await().until(() -> future1.isDone() && future2.isDone()); - assertThat(future1.get(), equalTo(1)); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future3 = identityLoader.load("C"); + identityLoader.dispatch(); + + await().until(() -> future1a.isDone() && future3.isDone()); + assertThat(future1a.get(), equalTo("A")); + assertThat(future3.get(), equalTo("C")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), singletonList("C")))); + + CompletableFuture future1b = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + CompletableFuture future3a = identityLoader.load("C"); + identityLoader.dispatch(); + + await().until(() -> future1b.isDone() && future2a.isDone() && future3a.isDone()); + assertThat(future1b.get(), equalTo("A")); + assertThat(future2a.get(), equalTo("B")); + assertThat(future3a.get(), equalTo("C")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), singletonList("C")))); + } + + @Test + public void should_Not_redispatch_previous_load() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + identityLoader.dispatch(); + + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(asList(singletonList("A"), singletonList("B")))); + } + + @Test + public void should_Cache_on_redispatch() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + identityLoader.dispatch(); + + CompletableFuture> future2 = identityLoader.loadMany(asList("A", "B")); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo(asList("A", "B"))); + assertThat(loadCalls, equalTo(asList(singletonList("A"), singletonList("B")))); + } + + @Test + public void should_Clear_single_value_in_loader() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); + + // fluency + DataLoader dl = identityLoader.clear("A"); + assertThat(dl, equalTo(identityLoader)); + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1a.isDone() && future2a.isDone()); + assertThat(future1a.get(), equalTo("A")); + assertThat(future2a.get(), equalTo("B")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), singletonList("A")))); + } + + @Test + public void should_Clear_all_values_in_loader() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); + + DataLoader dlFluent = identityLoader.clearAll(); + assertThat(dlFluent, equalTo(identityLoader)); // fluency + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1a.isDone() && future2a.isDone()); + assertThat(future1a.get(), equalTo("A")); + assertThat(future2a.get(), equalTo("B")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), asList("A", "B")))); + } + + @Test + public void should_Allow_priming_the_cache() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + DataLoader dlFluency = identityLoader.prime("A", "A"); + assertThat(dlFluency, equalTo(identityLoader)); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(singletonList("B")))); + } + + @Test + public void should_Not_prime_keys_that_already_exist() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + identityLoader.prime("A", "X"); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture> composite = identityLoader.dispatch(); + + await().until(composite::isDone); + assertThat(future1.get(), equalTo("X")); + assertThat(future2.get(), equalTo("B")); + + identityLoader.prime("A", "Y"); + identityLoader.prime("B", "Y"); + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + CompletableFuture> composite2 = identityLoader.dispatch(); + + await().until(composite2::isDone); + assertThat(future1a.get(), equalTo("X")); + assertThat(future2a.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(singletonList("B")))); + } + + @Test + public void should_Allow_to_forcefully_prime_the_cache() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + identityLoader.prime("A", "X"); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture> composite = identityLoader.dispatch(); + + await().until(composite::isDone); + assertThat(future1.get(), equalTo("X")); + assertThat(future2.get(), equalTo("B")); + + identityLoader.clear("A").prime("A", "Y"); + identityLoader.clear("B").prime("B", "Y"); + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + CompletableFuture> composite2 = identityLoader.dispatch(); + + await().until(composite2::isDone); + assertThat(future1a.get(), equalTo("Y")); + assertThat(future2a.get(), equalTo("Y")); + assertThat(loadCalls, equalTo(singletonList(singletonList("B")))); + } + + @Test + public void should_Allow_priming_the_cache_with_a_future() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + DataLoader dlFluency = identityLoader.prime("A", CompletableFuture.completedFuture("A")); + assertThat(dlFluency, equalTo(identityLoader)); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(singletonList("B")))); + } + + @Test + public void should_not_Cache_failed_fetches_on_complete_failure() { + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idLoaderBlowsUps(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + + CompletableFuture future2 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future2::isDone); + assertThat(future2.isCompletedExceptionally(), is(true)); + assertThat(cause(future2), instanceOf(IllegalStateException.class)); + assertThat(loadCalls, equalTo(asList(singletonList(1), singletonList(1)))); + } + + @Test + public void should_Resolve_to_error_to_indicate_failure() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader evenLoader = idLoaderOddEvenExceptions(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = evenLoader.load(1); + evenLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + + CompletableFuture future2 = evenLoader.load(2); + evenLoader.dispatch(); + + await().until(future2::isDone); assertThat(future2.get(), equalTo(2)); + assertThat(loadCalls, equalTo(asList(singletonList(1), singletonList(2)))); + } + + // Accept any kind of key. + + @Test + public void should_Represent_failures_and_successes_simultaneously() throws ExecutionException, InterruptedException { + AtomicBoolean success = new AtomicBoolean(); + List> loadCalls = new ArrayList<>(); + DataLoader evenLoader = idLoaderOddEvenExceptions(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = evenLoader.load(1); + CompletableFuture future2 = evenLoader.load(2); + CompletableFuture future3 = evenLoader.load(3); + CompletableFuture future4 = evenLoader.load(4); + CompletableFuture> result = evenLoader.dispatch(); + result.thenAccept(promisedValues -> success.set(true)); + + await().untilAtomic(success, is(true)); + + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + assertThat(future2.get(), equalTo(2)); + assertThat(future3.isCompletedExceptionally(), is(true)); + assertThat(future4.get(), equalTo(4)); + + assertThat(loadCalls, equalTo(singletonList(asList(1, 2, 3, 4)))); + } + + // Accepts options + + @Test + public void should_Cache_failed_fetches() { + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idLoaderAllExceptions(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + + CompletableFuture future2 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future2::isDone); + assertThat(future2.isCompletedExceptionally(), is(true)); + assertThat(cause(future2), instanceOf(IllegalStateException.class)); + + assertThat(loadCalls, equalTo(singletonList(singletonList(1)))); + } + + @Test + public void should_NOT_Cache_failed_fetches_if_told_not_too() { + DataLoaderOptions options = DataLoaderOptions.newOptions().setCachingExceptionsEnabled(false); + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idLoaderAllExceptions(options, loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + + CompletableFuture future2 = errorLoader.load(1); + errorLoader.dispatch(); + + await().until(future2::isDone); + assertThat(future2.isCompletedExceptionally(), is(true)); + assertThat(cause(future2), instanceOf(IllegalStateException.class)); + + assertThat(loadCalls, equalTo(asList(singletonList(1), singletonList(1)))); + } + + + // Accepts object key in custom cacheKey function + + @Test + public void should_Handle_priming_the_cache_with_an_error() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + identityLoader.prime(1, new IllegalStateException("Error")); + + CompletableFuture future1 = identityLoader.load(1); + identityLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + assertThat(loadCalls, equalTo(emptyList())); + } + + @Test + public void should_Clear_values_from_cache_after_errors() { + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idLoaderBlowsUps(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + future1.handle((value, t) -> { + if (t != null) { + // Presumably determine if this error is transient, and only clear the cache in that case. + errorLoader.clear(1); + } + return null; + }); + errorLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + assertThat(cause(future1), instanceOf(IllegalStateException.class)); + + CompletableFuture future2 = errorLoader.load(1); + future2.handle((value, t) -> { + if (t != null) { + // Again, only do this if you can determine the error is transient. + errorLoader.clear(1); + } + return null; + }); + errorLoader.dispatch(); + + await().until(future2::isDone); + assertThat(future2.isCompletedExceptionally(), is(true)); + assertThat(cause(future2), instanceOf(IllegalStateException.class)); + assertThat(loadCalls, equalTo(asList(singletonList(1), singletonList(1)))); + } + + @Test + public void should_Propagate_error_to_all_loads() { + List> loadCalls = new ArrayList<>(); + DataLoader errorLoader = idLoaderBlowsUps(new DataLoaderOptions(), loadCalls); + + CompletableFuture future1 = errorLoader.load(1); + CompletableFuture future2 = errorLoader.load(2); + errorLoader.dispatch(); + + await().until(future1::isDone); + assertThat(future1.isCompletedExceptionally(), is(true)); + Throwable cause = cause(future1); + assert cause != null; + assertThat(cause, instanceOf(IllegalStateException.class)); + assertThat(cause.getMessage(), equalTo("Error")); + + await().until(future2::isDone); + cause = cause(future2); + assert cause != null; + assertThat(cause.getMessage(), equalTo(cause.getMessage())); + assertThat(loadCalls, equalTo(singletonList(asList(1, 2)))); + } + + @Test + public void should_Accept_objects_as_keys() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(new DataLoaderOptions(), loadCalls); + + Object keyA = new Object(); + Object keyB = new Object(); + + // Fetches as expected + + identityLoader.load(keyA); + identityLoader.load(keyB); + + identityLoader.dispatch().thenAccept(promisedValues -> { + assertThat(promisedValues.get(0), equalTo(keyA)); + assertThat(promisedValues.get(1), equalTo(keyB)); + }); + + assertThat(loadCalls.size(), equalTo(1)); + assertThat(loadCalls.get(0).size(), equalTo(2)); + assertThat(loadCalls.get(0).toArray()[0], equalTo(keyA)); + assertThat(loadCalls.get(0).toArray()[1], equalTo(keyB)); + + // Caching + identityLoader.clear(keyA); + //noinspection SuspiciousMethodCalls + loadCalls.remove(keyA); + + identityLoader.load(keyA); + identityLoader.load(keyB); + + identityLoader.dispatch().thenAccept(promisedValues -> { + assertThat(promisedValues.get(0), equalTo(keyA)); + assertThat(identityLoader.getCacheKey(keyB), equalTo(keyB)); + }); + + assertThat(loadCalls.size(), equalTo(2)); + assertThat(loadCalls.get(1).size(), equalTo(1)); + assertThat(loadCalls.get(1).toArray()[0], equalTo(keyA)); + } + + @Test + public void should_Disable_caching() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = + idLoader(newOptions().setCachingEnabled(false), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); + + CompletableFuture future1a = identityLoader.load("A"); + CompletableFuture future3 = identityLoader.load("C"); + identityLoader.dispatch(); + + await().until(() -> future1a.isDone() && future3.isDone()); + assertThat(future1a.get(), equalTo("A")); + assertThat(future3.get(), equalTo("C")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), asList("A", "C")))); + + CompletableFuture future1b = identityLoader.load("A"); + CompletableFuture future2a = identityLoader.load("B"); + CompletableFuture future3a = identityLoader.load("C"); + identityLoader.dispatch(); + + await().until(() -> future1b.isDone() && future2a.isDone() && future3a.isDone()); + assertThat(future1b.get(), equalTo("A")); + assertThat(future2a.get(), equalTo("B")); + assertThat(future3a.get(), equalTo("C")); + assertThat(loadCalls, equalTo(asList(asList("A", "B"), + asList("A", "C"), asList("A", "B", "C")))); + } + + @Test + public void should_work_with_duplicate_keys_when_caching_disabled() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = + idLoader(newOptions().setCachingEnabled(false), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture future3 = identityLoader.load("A"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone() && future3.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(future3.get(), equalTo("A")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B", "A")))); + } + + @Test + public void should_work_with_duplicate_keys_when_caching_enabled() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = + idLoader(newOptions().setCachingEnabled(true), loadCalls); + + CompletableFuture future1 = identityLoader.load("A"); + CompletableFuture future2 = identityLoader.load("B"); + CompletableFuture future3 = identityLoader.load("A"); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone() && future3.isDone()); + assertThat(future1.get(), equalTo("A")); + assertThat(future2.get(), equalTo("B")); + assertThat(future3.get(), equalTo("A")); + assertThat(loadCalls, equalTo(singletonList(asList("A", "B")))); + } + + // It is resilient to job queue ordering + + @Test + public void should_Accept_objects_with_a_complex_key() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setCacheKeyFunction(getJsonObjectCacheMapFn()); + DataLoader identityLoader = idLoader(options, loadCalls); + + JsonObject key1 = new JsonObject().put("id", 123); + JsonObject key2 = new JsonObject().put("id", 123); + + CompletableFuture future1 = identityLoader.load(key1); + CompletableFuture future2 = identityLoader.load(key2); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(loadCalls, equalTo(singletonList(singletonList(key1)))); + assertThat(future1.get(), equalTo(key1)); + assertThat(future2.get(), equalTo(key1)); + } + + // Helper methods + + @Test + public void should_Clear_objects_with_complex_key() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setCacheKeyFunction(getJsonObjectCacheMapFn()); + DataLoader identityLoader = idLoader(options, loadCalls); + + JsonObject key1 = new JsonObject().put("id", 123); + JsonObject key2 = new JsonObject().put("id", 123); + + CompletableFuture future1 = identityLoader.load(key1); + identityLoader.dispatch(); + + await().until(future1::isDone); + identityLoader.clear(key2); // clear equivalent object key + + CompletableFuture future2 = identityLoader.load(key1); + identityLoader.dispatch(); + + await().until(future2::isDone); + assertThat(loadCalls, equalTo(asList(singletonList(key1), singletonList(key1)))); + assertThat(future1.get(), equalTo(key1)); + assertThat(future2.get(), equalTo(key1)); + } + + @Test + public void should_Accept_objects_with_different_order_of_keys() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setCacheKeyFunction(getJsonObjectCacheMapFn()); + DataLoader identityLoader = idLoader(options, loadCalls); + + JsonObject key1 = new JsonObject().put("a", 123).put("b", 321); + JsonObject key2 = new JsonObject().put("b", 321).put("a", 123); + + // Fetches as expected + + CompletableFuture future1 = identityLoader.load(key1); + CompletableFuture future2 = identityLoader.load(key2); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(loadCalls, equalTo(singletonList(singletonList(key1)))); + assertThat(loadCalls.size(), equalTo(1)); + assertThat(future1.get(), equalTo(key1)); + assertThat(future2.get(), equalTo(key2)); + } + + @Test + public void should_Allow_priming_the_cache_with_an_object_key() throws ExecutionException, InterruptedException { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setCacheKeyFunction(getJsonObjectCacheMapFn()); + DataLoader identityLoader = idLoader(options, loadCalls); + + JsonObject key1 = new JsonObject().put("id", 123); + JsonObject key2 = new JsonObject().put("id", 123); + + identityLoader.prime(key1, key1); + + CompletableFuture future1 = identityLoader.load(key1); + CompletableFuture future2 = identityLoader.load(key2); + identityLoader.dispatch(); + + await().until(() -> future1.isDone() && future2.isDone()); + assertThat(loadCalls, equalTo(emptyList())); + assertThat(future1.get(), equalTo(key1)); + assertThat(future2.get(), equalTo(key1)); + } + + @Test + public void should_Accept_a_custom_cache_map_implementation() throws ExecutionException, InterruptedException { + CustomCacheMap customMap = new CustomCacheMap(); + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setCacheMap(customMap); + DataLoader identityLoader = idLoader(options, loadCalls); + + // Fetches as expected + + CompletableFuture future1 = identityLoader.load("a"); + CompletableFuture future2 = identityLoader.load("b"); + CompletableFuture> composite = identityLoader.dispatch(); + + await().until(composite::isDone); + assertThat(future1.get(), equalTo("a")); + assertThat(future2.get(), equalTo("b")); + + assertThat(loadCalls, equalTo(singletonList(asList("a", "b")))); + assertArrayEquals(customMap.stash.keySet().toArray(), asList("a", "b").toArray()); + + CompletableFuture future3 = identityLoader.load("c"); + CompletableFuture future2a = identityLoader.load("b"); + composite = identityLoader.dispatch(); + + await().until(composite::isDone); + assertThat(future3.get(), equalTo("c")); + assertThat(future2a.get(), equalTo("b")); + + assertThat(loadCalls, equalTo(asList(asList("a", "b"), singletonList("c")))); + assertArrayEquals(customMap.stash.keySet().toArray(), asList("a", "b", "c").toArray()); + + // Supports clear + + identityLoader.clear("b"); + assertArrayEquals(customMap.stash.keySet().toArray(), asList("a", "c").toArray()); + + CompletableFuture future2b = identityLoader.load("b"); + composite = identityLoader.dispatch(); + + await().until(composite::isDone); + assertThat(future2b.get(), equalTo("b")); + assertThat(loadCalls, equalTo(asList(asList("a", "b"), + singletonList("c"), singletonList("b")))); + assertArrayEquals(customMap.stash.keySet().toArray(), asList("a", "c", "b").toArray()); + + // Supports clear all + + identityLoader.clearAll(); + assertArrayEquals(customMap.stash.keySet().toArray(), emptyList().toArray()); + } + + @Test + public void should_degrade_gracefully_if_cache_get_throws() { + CacheMap cache = new ThrowingCacheMap(); + DataLoaderOptions options = newOptions().setCachingEnabled(true).setCacheMap(cache); + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(options, loadCalls); + + assertThat(identityLoader.getIfPresent("a"), equalTo(Optional.empty())); + + CompletableFuture future = identityLoader.load("a"); + identityLoader.dispatch(); + assertThat(future.join(), equalTo("a")); + } + + @Test + public void batching_disabled_should_dispatch_immediately() { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setBatchingEnabled(false); + DataLoader identityLoader = idLoader(options, loadCalls); + + CompletableFuture fa = identityLoader.load("A"); + CompletableFuture fb = identityLoader.load("B"); + + // caching is on still + CompletableFuture fa1 = identityLoader.load("A"); + CompletableFuture fb1 = identityLoader.load("B"); + + List values = CompletableFutureKit.allOf(asList(fa, fb, fa1, fb1)).join(); + + assertThat(fa.join(), equalTo("A")); + assertThat(fb.join(), equalTo("B")); + assertThat(fa1.join(), equalTo("A")); + assertThat(fb1.join(), equalTo("B")); + + assertThat(values, equalTo(asList("A", "B", "A", "B"))); + + assertThat(loadCalls, equalTo(asList( + singletonList("A"), + singletonList("B")))); + + } + + @Test + public void batching_disabled_and_caching_disabled_should_dispatch_immediately_and_forget() { + List> loadCalls = new ArrayList<>(); + DataLoaderOptions options = newOptions().setBatchingEnabled(false).setCachingEnabled(false); + DataLoader identityLoader = idLoader(options, loadCalls); + + CompletableFuture fa = identityLoader.load("A"); + CompletableFuture fb = identityLoader.load("B"); + + // caching is off + CompletableFuture fa1 = identityLoader.load("A"); + CompletableFuture fb1 = identityLoader.load("B"); + + List values = CompletableFutureKit.allOf(asList(fa, fb, fa1, fb1)).join(); + + assertThat(fa.join(), equalTo("A")); + assertThat(fb.join(), equalTo("B")); + assertThat(fa1.join(), equalTo("A")); + assertThat(fb1.join(), equalTo("B")); + + assertThat(values, equalTo(asList("A", "B", "A", "B"))); + + assertThat(loadCalls, equalTo(asList( + singletonList("A"), + singletonList("B"), + singletonList("A"), + singletonList("B") + ))); + + } + + @Test + public void batches_multiple_requests_with_max_batch_size() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(newOptions().setMaxBatchSize(2), loadCalls); + + CompletableFuture f1 = identityLoader.load(1); + CompletableFuture f2 = identityLoader.load(2); + CompletableFuture f3 = identityLoader.load(3); + + identityLoader.dispatch(); + + CompletableFuture.allOf(f1, f2, f3).join(); + + assertThat(f1.join(), equalTo(1)); + assertThat(f2.join(), equalTo(2)); + assertThat(f3.join(), equalTo(3)); + + assertThat(loadCalls, equalTo(asList(asList(1, 2), singletonList(3)))); + + } + + @Test + public void can_split_max_batch_sizes_correctly() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(newOptions().setMaxBatchSize(5), loadCalls); + + for (int i = 0; i < 21; i++) { + identityLoader.load(i); + } + List> expectedCalls = new ArrayList<>(); + expectedCalls.add(listFrom(0, 5)); + expectedCalls.add(listFrom(5, 10)); + expectedCalls.add(listFrom(10, 15)); + expectedCalls.add(listFrom(15, 20)); + expectedCalls.add(listFrom(20, 21)); + + List result = identityLoader.dispatch().join(); + + assertThat(result, equalTo(listFrom(0, 21))); + assertThat(loadCalls, equalTo(expectedCalls)); + + } + + @Test + public void should_Batch_loads_occurring_within_futures() { + List> loadCalls = new ArrayList<>(); + DataLoader identityLoader = idLoader(newOptions(), loadCalls); + + Supplier nullValue = () -> null; + + AtomicBoolean v4Called = new AtomicBoolean(); + + CompletableFuture.supplyAsync(nullValue).thenAccept(v1 -> { + identityLoader.load("a"); + CompletableFuture.supplyAsync(nullValue).thenAccept(v2 -> { + identityLoader.load("b"); + CompletableFuture.supplyAsync(nullValue).thenAccept(v3 -> { + identityLoader.load("c"); + CompletableFuture.supplyAsync(nullValue).thenAccept( + v4 -> { + identityLoader.load("d"); + v4Called.set(true); + }); + }); + }); + }); + + await().untilTrue(v4Called); + + identityLoader.dispatchAndJoin(); + + assertThat(loadCalls, equalTo( + singletonList(asList("a", "b", "c", "d")))); + } + + @Test + public void can_call_a_loader_from_a_loader() throws Exception { + List> deepLoadCalls = new ArrayList<>(); + DataLoader deepLoader = newDataLoader(keys -> { + deepLoadCalls.add(keys); + return CompletableFuture.completedFuture(keys); + }); + + List> aLoadCalls = new ArrayList<>(); + DataLoader aLoader = newDataLoader(keys -> { + aLoadCalls.add(keys); + return deepLoader.loadMany(keys); + }); + + List> bLoadCalls = new ArrayList<>(); + DataLoader bLoader = newDataLoader(keys -> { + bLoadCalls.add(keys); + return deepLoader.loadMany(keys); + }); + + CompletableFuture a1 = aLoader.load("A1"); + CompletableFuture a2 = aLoader.load("A2"); + CompletableFuture b1 = bLoader.load("B1"); + CompletableFuture b2 = bLoader.load("B2"); + + CompletableFuture.allOf( + aLoader.dispatch(), + deepLoader.dispatch(), + bLoader.dispatch(), + deepLoader.dispatch() + ).join(); + + assertThat(a1.get(), equalTo("A1")); + assertThat(a2.get(), equalTo("A2")); + assertThat(b1.get(), equalTo("B1")); + assertThat(b2.get(), equalTo("B2")); + + assertThat(aLoadCalls, equalTo( + singletonList(asList("A1", "A2")))); + + assertThat(bLoadCalls, equalTo( + singletonList(asList("B1", "B2")))); + + assertThat(deepLoadCalls, equalTo( + asList(asList("A1", "A2"), asList("B1", "B2")))); + } + + @Test + public void should_allow_composition_of_data_loader_calls() { + UserManager userManager = new UserManager(); + + BatchLoader userBatchLoader = userIds -> CompletableFuture + .supplyAsync(() -> userIds + .stream() + .map(userManager::loadUserById) + .collect(Collectors.toList())); + DataLoader userLoader = newDataLoader(userBatchLoader); + + AtomicBoolean gandalfCalled = new AtomicBoolean(false); + AtomicBoolean sarumanCalled = new AtomicBoolean(false); + + userLoader.load(1L) + .thenAccept(user -> userLoader.load(user.getInvitedByID()) + .thenAccept(invitedBy -> { + gandalfCalled.set(true); + assertThat(invitedBy.getName(), equalTo("Manwë")); + })); + + userLoader.load(2L) + .thenAccept(user -> userLoader.load(user.getInvitedByID()) + .thenAccept(invitedBy -> { + sarumanCalled.set(true); + assertThat(invitedBy.getName(), equalTo("Aulë")); + })); + + List allResults = userLoader.dispatchAndJoin(); + + await().untilTrue(gandalfCalled); + await().untilTrue(sarumanCalled); + + assertThat(allResults.size(), equalTo(4)); + } + + private static CacheKey getJsonObjectCacheMapFn() { + return key -> key.stream() + .map(entry -> entry.getKey() + ":" + entry.getValue()) + .sorted() + .collect(Collectors.joining()); + } + + private static DataLoader idLoader(DataLoaderOptions options, List> loadCalls) { + return newPublisherDataLoader((PublisherBatchLoader) (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + Flux.fromIterable(keys).subscribe(subscriber); + }, options); + } + + private static DataLoader idLoaderBlowsUps( + DataLoaderOptions options, List> loadCalls) { + return newPublisherDataLoader((PublisherBatchLoader) (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + Flux.error(new IllegalStateException("Error")).subscribe(subscriber); + }, options); + } + + private static DataLoader idLoaderAllExceptions( + DataLoaderOptions options, List> loadCalls) { + return newPublisherDataLoaderWithTry((PublisherBatchLoader>) (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + Stream> failures = keys.stream().map(k -> Try.failed(new IllegalStateException("Error"))); + Flux.fromStream(failures).subscribe(subscriber); + }, options); + } + + private static DataLoader idLoaderOddEvenExceptions( + DataLoaderOptions options, List> loadCalls) { + return newPublisherDataLoaderWithTry((PublisherBatchLoader>) (keys, subscriber) -> { + loadCalls.add(new ArrayList<>(keys)); + + List> errors = new ArrayList<>(); + for (Integer key : keys) { + if (key % 2 == 0) { + errors.add(Try.succeeded(key)); + } else { + errors.add(Try.failed(new IllegalStateException("Error"))); + } + } + Flux.fromIterable(errors).subscribe(subscriber); + }, options); } private static PublisherBatchLoader keysAsValues() { return (keys, subscriber) -> Flux.fromIterable(keys).subscribe(subscriber); } + + private static class ThrowingCacheMap extends CustomCacheMap { + @Override + public CompletableFuture get(String key) { + throw new RuntimeException("Cache implementation failed."); + } + } } From 0d0b2f8b9626dd1b967c164029a37f213c481f91 Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sun, 19 May 2024 00:42:27 +1000 Subject: [PATCH 7/9] Rename '*PublisherBatchLoader' to 'BatchPublisher' This keeps in line with the original suggestion (because yours truly couldn't read, apparently). We also purge any remaining mention of 'observer', which was the first swing at this code. --- ...erBatchLoader.java => BatchPublisher.java} | 2 +- ...xt.java => BatchPublisherWithContext.java} | 4 +- .../org/dataloader/DataLoaderFactory.java | 36 ++++++------ .../java/org/dataloader/DataLoaderHelper.java | 56 +++++++++---------- ...hLoader.java => MappedBatchPublisher.java} | 2 +- ...a => MappedBatchPublisherWithContext.java} | 4 +- .../scheduler/BatchLoaderScheduler.java | 18 +++--- src/test/java/ReadmeExamples.java | 2 +- ...java => DataLoaderBatchPublisherTest.java} | 12 ++-- ...> DataLoaderMappedBatchPublisherTest.java} | 8 +-- .../scheduler/BatchLoaderSchedulerTest.java | 6 +- 11 files changed, 75 insertions(+), 75 deletions(-) rename src/main/java/org/dataloader/{PublisherBatchLoader.java => BatchPublisher.java} (94%) rename src/main/java/org/dataloader/{PublisherBatchLoaderWithContext.java => BatchPublisherWithContext.java} (51%) rename src/main/java/org/dataloader/{MappedPublisherBatchLoader.java => MappedBatchPublisher.java} (93%) rename src/main/java/org/dataloader/{MappedPublisherBatchLoaderWithContext.java => MappedBatchPublisherWithContext.java} (54%) rename src/test/java/org/dataloader/{DataLoaderPublisherBatchLoaderTest.java => DataLoaderBatchPublisherTest.java} (98%) rename src/test/java/org/dataloader/{DataLoaderMappedPublisherBatchLoaderTest.java => DataLoaderMappedBatchPublisherTest.java} (95%) diff --git a/src/main/java/org/dataloader/PublisherBatchLoader.java b/src/main/java/org/dataloader/BatchPublisher.java similarity index 94% rename from src/main/java/org/dataloader/PublisherBatchLoader.java rename to src/main/java/org/dataloader/BatchPublisher.java index 2dcdf1e..9d3932a 100644 --- a/src/main/java/org/dataloader/PublisherBatchLoader.java +++ b/src/main/java/org/dataloader/BatchPublisher.java @@ -16,6 +16,6 @@ * @param type parameter indicating the type of keys to use for data load requests. * @param type parameter indicating the type of values returned */ -public interface PublisherBatchLoader { +public interface BatchPublisher { void load(List keys, Subscriber subscriber); } diff --git a/src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java b/src/main/java/org/dataloader/BatchPublisherWithContext.java similarity index 51% rename from src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java rename to src/main/java/org/dataloader/BatchPublisherWithContext.java index 45ea36d..effda90 100644 --- a/src/main/java/org/dataloader/PublisherBatchLoaderWithContext.java +++ b/src/main/java/org/dataloader/BatchPublisherWithContext.java @@ -5,8 +5,8 @@ import java.util.List; /** - * An {@link PublisherBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + * An {@link BatchPublisher} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. */ -public interface PublisherBatchLoaderWithContext { +public interface BatchPublisherWithContext { void load(List keys, Subscriber subscriber, BatchLoaderEnvironment environment); } diff --git a/src/main/java/org/dataloader/DataLoaderFactory.java b/src/main/java/org/dataloader/DataLoaderFactory.java index 5b50874..db14f2e 100644 --- a/src/main/java/org/dataloader/DataLoaderFactory.java +++ b/src/main/java/org/dataloader/DataLoaderFactory.java @@ -288,7 +288,7 @@ public static DataLoader newMappedDataLoaderWithTry(MappedBatchLoad * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoader(PublisherBatchLoader batchLoadFunction) { + public static DataLoader newPublisherDataLoader(BatchPublisher batchLoadFunction) { return newPublisherDataLoader(batchLoadFunction, null); } @@ -302,7 +302,7 @@ public static DataLoader newPublisherDataLoader(PublisherBatchLoade * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoader(PublisherBatchLoader batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newPublisherDataLoader(BatchPublisher batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -323,7 +323,7 @@ public static DataLoader newPublisherDataLoader(PublisherBatchLoade * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoader> batchLoadFunction) { + public static DataLoader newPublisherDataLoaderWithTry(BatchPublisher> batchLoadFunction) { return newPublisherDataLoaderWithTry(batchLoadFunction, null); } @@ -341,7 +341,7 @@ public static DataLoader newPublisherDataLoaderWithTry(PublisherBat * * @see #newDataLoaderWithTry(BatchLoader) */ - public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoader> batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newPublisherDataLoaderWithTry(BatchPublisher> batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -355,7 +355,7 @@ public static DataLoader newPublisherDataLoaderWithTry(PublisherBat * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoader(PublisherBatchLoaderWithContext batchLoadFunction) { + public static DataLoader newPublisherDataLoader(BatchPublisherWithContext batchLoadFunction) { return newPublisherDataLoader(batchLoadFunction, null); } @@ -369,7 +369,7 @@ public static DataLoader newPublisherDataLoader(PublisherBatchLoade * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoader(PublisherBatchLoaderWithContext batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newPublisherDataLoader(BatchPublisherWithContext batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -390,7 +390,7 @@ public static DataLoader newPublisherDataLoader(PublisherBatchLoade * * @return a new DataLoader */ - public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoaderWithContext> batchLoadFunction) { + public static DataLoader newPublisherDataLoaderWithTry(BatchPublisherWithContext> batchLoadFunction) { return newPublisherDataLoaderWithTry(batchLoadFunction, null); } @@ -406,9 +406,9 @@ public static DataLoader newPublisherDataLoaderWithTry(PublisherBat * * @return a new DataLoader * - * @see #newPublisherDataLoaderWithTry(PublisherBatchLoader) + * @see #newPublisherDataLoaderWithTry(BatchPublisher) */ - public static DataLoader newPublisherDataLoaderWithTry(PublisherBatchLoaderWithContext> batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newPublisherDataLoaderWithTry(BatchPublisherWithContext> batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -422,7 +422,7 @@ public static DataLoader newPublisherDataLoaderWithTry(PublisherBat * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoader batchLoadFunction) { + public static DataLoader newMappedPublisherDataLoader(MappedBatchPublisher batchLoadFunction) { return newMappedPublisherDataLoader(batchLoadFunction, null); } @@ -436,7 +436,7 @@ public static DataLoader newMappedPublisherDataLoader(MappedPublish * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoader batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newMappedPublisherDataLoader(MappedBatchPublisher batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -457,7 +457,7 @@ public static DataLoader newMappedPublisherDataLoader(MappedPublish * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader> batchLoadFunction) { + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedBatchPublisher> batchLoadFunction) { return newMappedPublisherDataLoaderWithTry(batchLoadFunction, null); } @@ -475,7 +475,7 @@ public static DataLoader newMappedPublisherDataLoaderWithTry(Mapped * * @see #newDataLoaderWithTry(BatchLoader) */ - public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader> batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedBatchPublisher> batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -489,7 +489,7 @@ public static DataLoader newMappedPublisherDataLoaderWithTry(Mapped * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoaderWithContext batchLoadFunction) { + public static DataLoader newMappedPublisherDataLoader(MappedBatchPublisherWithContext batchLoadFunction) { return newMappedPublisherDataLoader(batchLoadFunction, null); } @@ -503,7 +503,7 @@ public static DataLoader newMappedPublisherDataLoader(MappedPublish * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoader(MappedPublisherBatchLoaderWithContext batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newMappedPublisherDataLoader(MappedBatchPublisherWithContext batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } @@ -524,7 +524,7 @@ public static DataLoader newMappedPublisherDataLoader(MappedPublish * * @return a new DataLoader */ - public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoaderWithContext> batchLoadFunction) { + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedBatchPublisherWithContext> batchLoadFunction) { return newMappedPublisherDataLoaderWithTry(batchLoadFunction, null); } @@ -540,9 +540,9 @@ public static DataLoader newMappedPublisherDataLoaderWithTry(Mapped * * @return a new DataLoader * - * @see #newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoader) + * @see #newMappedPublisherDataLoaderWithTry(MappedBatchPublisher) */ - public static DataLoader newMappedPublisherDataLoaderWithTry(MappedPublisherBatchLoaderWithContext> batchLoadFunction, DataLoaderOptions options) { + public static DataLoader newMappedPublisherDataLoaderWithTry(MappedBatchPublisherWithContext> batchLoadFunction, DataLoaderOptions options) { return mkDataLoader(batchLoadFunction, options); } diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index 12817e1..2cb6a7f 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -248,7 +248,7 @@ private CompletableFuture> dispatchQueueBatch(List keys, List return batchLoad .thenApply(values -> { assertResultSize(keys, values); - if (isPublisherLoader() || isMappedPublisherLoader()) { + if (isPublisher() || isMappedPublisher()) { // We have already completed the queued futures by the time the overall batchLoad future has completed. return values; } @@ -430,10 +430,10 @@ CompletableFuture> invokeLoader(List keys, List keyContexts, .context(context).keyContexts(keys, keyContexts).build(); if (isMapLoader()) { batchLoad = invokeMapBatchLoader(keys, environment); - } else if (isPublisherLoader()) { - batchLoad = invokePublisherBatchLoader(keys, keyContexts, queuedFutures, environment); - } else if (isMappedPublisherLoader()) { - batchLoad = invokeMappedPublisherBatchLoader(keys, keyContexts, queuedFutures, environment); + } else if (isPublisher()) { + batchLoad = invokeBatchPublisher(keys, keyContexts, queuedFutures, environment); + } else if (isMappedPublisher()) { + batchLoad = invokeMappedBatchPublisher(keys, keyContexts, queuedFutures, environment); } else { batchLoad = invokeListBatchLoader(keys, environment); } @@ -505,24 +505,24 @@ private CompletableFuture> invokeMapBatchLoader(List keys, BatchLoade }); } - private CompletableFuture> invokePublisherBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + private CompletableFuture> invokeBatchPublisher(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { CompletableFuture> loadResult = new CompletableFuture<>(); Subscriber subscriber = new DataLoaderSubscriber(loadResult, keys, keyContexts, queuedFutures); BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); - if (batchLoadFunction instanceof PublisherBatchLoaderWithContext) { - PublisherBatchLoaderWithContext loadFunction = (PublisherBatchLoaderWithContext) batchLoadFunction; + if (batchLoadFunction instanceof BatchPublisherWithContext) { + BatchPublisherWithContext loadFunction = (BatchPublisherWithContext) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, subscriber, environment); - batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); + BatchLoaderScheduler.ScheduledBatchPublisherCall loadCall = () -> loadFunction.load(keys, subscriber, environment); + batchLoaderScheduler.scheduleBatchPublisher(loadCall, keys, environment); } else { loadFunction.load(keys, subscriber, environment); } } else { - PublisherBatchLoader loadFunction = (PublisherBatchLoader) batchLoadFunction; + BatchPublisher loadFunction = (BatchPublisher) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, subscriber); - batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); + BatchLoaderScheduler.ScheduledBatchPublisherCall loadCall = () -> loadFunction.load(keys, subscriber); + batchLoaderScheduler.scheduleBatchPublisher(loadCall, keys, null); } else { loadFunction.load(keys, subscriber); } @@ -530,26 +530,26 @@ private CompletableFuture> invokePublisherBatchLoader(List keys, List return loadResult; } - private CompletableFuture> invokeMappedPublisherBatchLoader(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { + private CompletableFuture> invokeMappedBatchPublisher(List keys, List keyContexts, List> queuedFutures, BatchLoaderEnvironment environment) { CompletableFuture> loadResult = new CompletableFuture<>(); - Subscriber> observer = new DataLoaderMapEntrySubscriber(loadResult, keys, keyContexts, queuedFutures); + Subscriber> subscriber = new DataLoaderMapEntrySubscriber(loadResult, keys, keyContexts, queuedFutures); BatchLoaderScheduler batchLoaderScheduler = loaderOptions.getBatchLoaderScheduler(); - if (batchLoadFunction instanceof MappedPublisherBatchLoaderWithContext) { - MappedPublisherBatchLoaderWithContext loadFunction = (MappedPublisherBatchLoaderWithContext) batchLoadFunction; + if (batchLoadFunction instanceof MappedBatchPublisherWithContext) { + MappedBatchPublisherWithContext loadFunction = (MappedBatchPublisherWithContext) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer, environment); - batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, environment); + BatchLoaderScheduler.ScheduledBatchPublisherCall loadCall = () -> loadFunction.load(keys, subscriber, environment); + batchLoaderScheduler.scheduleBatchPublisher(loadCall, keys, environment); } else { - loadFunction.load(keys, observer, environment); + loadFunction.load(keys, subscriber, environment); } } else { - MappedPublisherBatchLoader loadFunction = (MappedPublisherBatchLoader) batchLoadFunction; + MappedBatchPublisher loadFunction = (MappedBatchPublisher) batchLoadFunction; if (batchLoaderScheduler != null) { - BatchLoaderScheduler.ScheduledObserverBatchLoaderCall loadCall = () -> loadFunction.load(keys, observer); - batchLoaderScheduler.scheduleObserverBatchLoader(loadCall, keys, null); + BatchLoaderScheduler.ScheduledBatchPublisherCall loadCall = () -> loadFunction.load(keys, subscriber); + batchLoaderScheduler.scheduleBatchPublisher(loadCall, keys, null); } else { - loadFunction.load(keys, observer); + loadFunction.load(keys, subscriber); } } return loadResult; @@ -559,12 +559,12 @@ private boolean isMapLoader() { return batchLoadFunction instanceof MappedBatchLoader || batchLoadFunction instanceof MappedBatchLoaderWithContext; } - private boolean isPublisherLoader() { - return batchLoadFunction instanceof PublisherBatchLoader; + private boolean isPublisher() { + return batchLoadFunction instanceof BatchPublisher; } - private boolean isMappedPublisherLoader() { - return batchLoadFunction instanceof MappedPublisherBatchLoader; + private boolean isMappedPublisher() { + return batchLoadFunction instanceof MappedBatchPublisher; } int dispatchDepth() { diff --git a/src/main/java/org/dataloader/MappedPublisherBatchLoader.java b/src/main/java/org/dataloader/MappedBatchPublisher.java similarity index 93% rename from src/main/java/org/dataloader/MappedPublisherBatchLoader.java rename to src/main/java/org/dataloader/MappedBatchPublisher.java index 9c7430a..9b3fcb9 100644 --- a/src/main/java/org/dataloader/MappedPublisherBatchLoader.java +++ b/src/main/java/org/dataloader/MappedBatchPublisher.java @@ -15,6 +15,6 @@ * @param type parameter indicating the type of keys to use for data load requests. * @param type parameter indicating the type of values returned */ -public interface MappedPublisherBatchLoader { +public interface MappedBatchPublisher { void load(List keys, Subscriber> subscriber); } diff --git a/src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java b/src/main/java/org/dataloader/MappedBatchPublisherWithContext.java similarity index 54% rename from src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java rename to src/main/java/org/dataloader/MappedBatchPublisherWithContext.java index a752abc..4810111 100644 --- a/src/main/java/org/dataloader/MappedPublisherBatchLoaderWithContext.java +++ b/src/main/java/org/dataloader/MappedBatchPublisherWithContext.java @@ -6,8 +6,8 @@ import java.util.Map; /** - * A {@link MappedPublisherBatchLoader} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. + * A {@link MappedBatchPublisher} with a {@link BatchLoaderEnvironment} provided as an extra parameter to {@link #load}. */ -public interface MappedPublisherBatchLoaderWithContext { +public interface MappedBatchPublisherWithContext { void load(List keys, Subscriber> subscriber, BatchLoaderEnvironment environment); } diff --git a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java index 2e82eff..e7e95d9 100644 --- a/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java +++ b/src/main/java/org/dataloader/scheduler/BatchLoaderScheduler.java @@ -5,8 +5,8 @@ import org.dataloader.DataLoader; import org.dataloader.DataLoaderOptions; import org.dataloader.MappedBatchLoader; -import org.dataloader.MappedPublisherBatchLoader; -import org.dataloader.PublisherBatchLoader; +import org.dataloader.MappedBatchPublisher; +import org.dataloader.BatchPublisher; import java.util.List; import java.util.Map; @@ -45,9 +45,9 @@ interface ScheduledMappedBatchLoaderCall { } /** - * This represents a callback that will invoke a {@link PublisherBatchLoader} or {@link MappedPublisherBatchLoader} function under the covers + * This represents a callback that will invoke a {@link BatchPublisher} or {@link MappedBatchPublisher} function under the covers */ - interface ScheduledObserverBatchLoaderCall { + interface ScheduledBatchPublisherCall { void invoke(); } @@ -82,14 +82,14 @@ interface ScheduledObserverBatchLoaderCall { CompletionStage> scheduleMappedBatchLoader(ScheduledMappedBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); /** - * This is called to schedule a {@link PublisherBatchLoader} call. + * This is called to schedule a {@link BatchPublisher} call. * - * @param scheduledCall the callback that needs to be invoked to allow the {@link PublisherBatchLoader} to proceed. - * @param keys this is the list of keys that will be passed to the {@link PublisherBatchLoader}. + * @param scheduledCall the callback that needs to be invoked to allow the {@link BatchPublisher} to proceed. + * @param keys this is the list of keys that will be passed to the {@link BatchPublisher}. * This is provided only for informative reasons and, you can't change the keys that are used * @param environment this is the {@link BatchLoaderEnvironment} in place, - * which can be null if it's a simple {@link PublisherBatchLoader} call + * which can be null if it's a simple {@link BatchPublisher} call * @param the key type */ - void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment); + void scheduleBatchPublisher(ScheduledBatchPublisherCall scheduledCall, List keys, BatchLoaderEnvironment environment); } diff --git a/src/test/java/ReadmeExamples.java b/src/test/java/ReadmeExamples.java index df733ed..31354ea 100644 --- a/src/test/java/ReadmeExamples.java +++ b/src/test/java/ReadmeExamples.java @@ -306,7 +306,7 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp } @Override - public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + public void scheduleBatchPublisher(ScheduledBatchPublisherCall scheduledCall, List keys, BatchLoaderEnvironment environment) { snooze(10); scheduledCall.invoke(); } diff --git a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderBatchPublisherTest.java similarity index 98% rename from src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java rename to src/test/java/org/dataloader/DataLoaderBatchPublisherTest.java index 508a031..84a8b18 100644 --- a/src/test/java/org/dataloader/DataLoaderPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderBatchPublisherTest.java @@ -37,7 +37,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertThat; -public class DataLoaderPublisherBatchLoaderTest { +public class DataLoaderBatchPublisherTest { @Test public void should_Build_a_really_really_simple_data_loader() { @@ -1043,7 +1043,7 @@ private static CacheKey getJsonObjectCacheMapFn() { } private static DataLoader idLoader(DataLoaderOptions options, List> loadCalls) { - return newPublisherDataLoader((PublisherBatchLoader) (keys, subscriber) -> { + return newPublisherDataLoader((BatchPublisher) (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); Flux.fromIterable(keys).subscribe(subscriber); }, options); @@ -1051,7 +1051,7 @@ private static DataLoader idLoader(DataLoaderOptions options, List DataLoader idLoaderBlowsUps( DataLoaderOptions options, List> loadCalls) { - return newPublisherDataLoader((PublisherBatchLoader) (keys, subscriber) -> { + return newPublisherDataLoader((BatchPublisher) (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); Flux.error(new IllegalStateException("Error")).subscribe(subscriber); }, options); @@ -1059,7 +1059,7 @@ private static DataLoader idLoaderBlowsUps( private static DataLoader idLoaderAllExceptions( DataLoaderOptions options, List> loadCalls) { - return newPublisherDataLoaderWithTry((PublisherBatchLoader>) (keys, subscriber) -> { + return newPublisherDataLoaderWithTry((BatchPublisher>) (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); Stream> failures = keys.stream().map(k -> Try.failed(new IllegalStateException("Error"))); Flux.fromStream(failures).subscribe(subscriber); @@ -1068,7 +1068,7 @@ private static DataLoader idLoaderAllExceptions( private static DataLoader idLoaderOddEvenExceptions( DataLoaderOptions options, List> loadCalls) { - return newPublisherDataLoaderWithTry((PublisherBatchLoader>) (keys, subscriber) -> { + return newPublisherDataLoaderWithTry((BatchPublisher>) (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); List> errors = new ArrayList<>(); @@ -1083,7 +1083,7 @@ private static DataLoader idLoaderOddEvenExceptions( }, options); } - private static PublisherBatchLoader keysAsValues() { + private static BatchPublisher keysAsValues() { return (keys, subscriber) -> Flux.fromIterable(keys).subscribe(subscriber); } diff --git a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java b/src/test/java/org/dataloader/DataLoaderMappedBatchPublisherTest.java similarity index 95% rename from src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java rename to src/test/java/org/dataloader/DataLoaderMappedBatchPublisherTest.java index 8e33300..c16c58f 100644 --- a/src/test/java/org/dataloader/DataLoaderMappedPublisherBatchLoaderTest.java +++ b/src/test/java/org/dataloader/DataLoaderMappedBatchPublisherTest.java @@ -24,9 +24,9 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -public class DataLoaderMappedPublisherBatchLoaderTest { +public class DataLoaderMappedBatchPublisherTest { - MappedPublisherBatchLoader evensOnlyMappedBatchLoader = (keys, subscriber) -> { + MappedBatchPublisher evensOnlyMappedBatchLoader = (keys, subscriber) -> { Map mapOfResults = new HashMap<>(); AtomicInteger index = new AtomicInteger(); @@ -40,7 +40,7 @@ public class DataLoaderMappedPublisherBatchLoaderTest { }; private static DataLoader idMapLoader(DataLoaderOptions options, List> loadCalls) { - MappedPublisherBatchLoader kvBatchLoader = (keys, subscriber) -> { + MappedBatchPublisher kvBatchLoader = (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); Map map = new HashMap<>(); //noinspection unchecked @@ -52,7 +52,7 @@ private static DataLoader idMapLoader(DataLoaderOptions options, Li private static DataLoader idMapLoaderBlowsUps( DataLoaderOptions options, List> loadCalls) { - return newMappedPublisherDataLoader((MappedPublisherBatchLoader) (keys, subscriber) -> { + return newMappedPublisherDataLoader((MappedBatchPublisher) (keys, subscriber) -> { loadCalls.add(new ArrayList<>(keys)); Flux.>error(new IllegalStateException("Error")).subscribe(subscriber); }, options); diff --git a/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java b/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java index b77026c..e9c43f8 100644 --- a/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java +++ b/src/test/java/org/dataloader/scheduler/BatchLoaderSchedulerTest.java @@ -38,7 +38,7 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp } @Override - public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + public void scheduleBatchPublisher(ScheduledBatchPublisherCall scheduledCall, List keys, BatchLoaderEnvironment environment) { scheduledCall.invoke(); } }; @@ -63,7 +63,7 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp } @Override - public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + public void scheduleBatchPublisher(ScheduledBatchPublisherCall scheduledCall, List keys, BatchLoaderEnvironment environment) { snooze(ms); scheduledCall.invoke(); } @@ -152,7 +152,7 @@ public CompletionStage> scheduleMappedBatchLoader(ScheduledMapp } @Override - public void scheduleObserverBatchLoader(ScheduledObserverBatchLoaderCall scheduledCall, List keys, BatchLoaderEnvironment environment) { + public void scheduleBatchPublisher(ScheduledBatchPublisherCall scheduledCall, List keys, BatchLoaderEnvironment environment) { CompletableFuture.supplyAsync(() -> { snooze(10); scheduledCall.invoke(); From 14002f6097ef599529f301818b250883d5b0b817 Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sun, 19 May 2024 00:51:47 +1000 Subject: [PATCH 8/9] Ensure DataLoaderSubscriber is only called by one thread Multiple threads may call `onNext` - we thus (lazily) chuck a `synchronized` to ensure correctness at the cost of speed. In future, we should examine how we should manage this concurrency better. --- src/main/java/org/dataloader/DataLoaderHelper.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/dataloader/DataLoaderHelper.java b/src/main/java/org/dataloader/DataLoaderHelper.java index 2cb6a7f..ee8d78b 100644 --- a/src/main/java/org/dataloader/DataLoaderHelper.java +++ b/src/main/java/org/dataloader/DataLoaderHelper.java @@ -648,8 +648,10 @@ public void onSubscribe(Subscription subscription) { subscription.request(keys.size()); } + // onNext may be called by multiple threads - for the time being, we pass 'synchronized' to guarantee + // correctness (at the cost of speed). @Override - public void onNext(V value) { + public synchronized void onNext(V value) { assertState(!onErrorCalled, () -> "onError has already been called; onNext may not be invoked."); assertState(!onCompleteCalled, () -> "onComplete has already been called; onNext may not be invoked."); From 0f303a83707f3443b49fbd0e281a8d395f032c6f Mon Sep 17 00:00:00 2001 From: Alexandre Carlton Date: Sun, 19 May 2024 00:55:17 +1000 Subject: [PATCH 9/9] Document Subscriber#onNext invocation order --- src/main/java/org/dataloader/BatchPublisher.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/dataloader/BatchPublisher.java b/src/main/java/org/dataloader/BatchPublisher.java index 9d3932a..5ab41e1 100644 --- a/src/main/java/org/dataloader/BatchPublisher.java +++ b/src/main/java/org/dataloader/BatchPublisher.java @@ -11,7 +11,8 @@ * the future returned by {@link DataLoader#load(Object)} to complete as soon as the individual value is available * (rather than when all values have been retrieved). *

- * It is required that values be returned in the same order as the keys provided. + * NOTE: It is required that {@link Subscriber#onNext(V)} is invoked on each value in the same order as + * the provided keys. * * @param type parameter indicating the type of keys to use for data load requests. * @param type parameter indicating the type of values returned