diff --git a/src/main/java/com/basho/riak/client/api/AsIsRiakCommand.java b/src/main/java/com/basho/riak/client/api/AsIsRiakCommand.java new file mode 100644 index 000000000..b21dd82c5 --- /dev/null +++ b/src/main/java/com/basho/riak/client/api/AsIsRiakCommand.java @@ -0,0 +1,37 @@ +/* + * Copyright 2013-2016 Basho Technologies Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.basho.riak.client.api; + +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.PBStreamingFutureOperation; +import com.basho.riak.client.core.RiakCluster; +import com.basho.riak.client.core.RiakFuture; + +/** + * @author Sergey Galkin + * @since 2.1.0 + */ +public abstract class AsIsRiakCommand extends RiakCommand +{ + protected abstract FutureOperation buildCoreOperation(); + + protected RiakFuture executeAsync(RiakCluster cluster) + { + final FutureOperation coreOperation = buildCoreOperation(); + + return cluster.execute(coreOperation); + } +} diff --git a/src/main/java/com/basho/riak/client/api/GenericRiakCommand.java b/src/main/java/com/basho/riak/client/api/GenericRiakCommand.java new file mode 100644 index 000000000..18cc53cbd --- /dev/null +++ b/src/main/java/com/basho/riak/client/api/GenericRiakCommand.java @@ -0,0 +1,70 @@ +/* + * Copyright 2013-2016 Basho Technologies Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.basho.riak.client.api; + +import com.basho.riak.client.api.commands.CoreFutureAdapter; +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.RiakCluster; +import com.basho.riak.client.core.RiakFuture; + +/** + * @author Sergey Galkin + * @since 2.1.0 + */ +public abstract class GenericRiakCommand extends RiakCommand +{ + public static abstract class GenericRiakCommandWithSameInfo extends GenericRiakCommand + { + @Override + protected I convertInfo(I coreInfo) { + return coreInfo; + } + } + + protected abstract FutureOperation buildCoreOperation(); + + protected RiakFuture executeAsync(RiakCluster cluster) + { + final FutureOperation coreOperation = buildCoreOperation(); + assert coreOperation != null; + + final RiakFuture coreFuture = cluster.execute(coreOperation); + + assert coreFuture != null; + + final CoreFutureAdapter future = + new CoreFutureAdapter(coreFuture) + { + @Override + protected R convertResponse(CoreR coreResponse) + { + return GenericRiakCommand.this.convertResponse(coreOperation, coreResponse); + } + + @Override + protected I convertQueryInfo(CoreI coreQueryInfo) + { + return GenericRiakCommand.this.convertInfo(coreQueryInfo); + } + }; + coreFuture.addListener(future); + return future; + } + + protected abstract R convertResponse(FutureOperation request, CoreR coreResponse); + + protected abstract I convertInfo(CoreI coreInfo); +} diff --git a/src/main/java/com/basho/riak/client/api/RiakClient.java b/src/main/java/com/basho/riak/client/api/RiakClient.java index 114c31b38..2146beef8 100644 --- a/src/main/java/com/basho/riak/client/api/RiakClient.java +++ b/src/main/java/com/basho/riak/client/api/RiakClient.java @@ -155,6 +155,7 @@ * * @author Dave Rusek * @author Brian Roach + * @author Alex Moore * @author Sergey Galkin * @since 2.0 */ @@ -277,8 +278,9 @@ public static RiakClient newClient(InetSocketAddress... addresses) throws Unknow * @return a new RiakClient instance. * @throws java.net.UnknownHostException if a supplied hostname cannot be resolved. * @since 2.0.3 - * @see com.basho.riak.client.core.RiakCluster.Builder#Builder(RiakNode.Builder, List) + * @see com.basho.riak.client.core.RiakCluster.Builder#RiakCluster.Builder(RiakNode.Builder, List) */ + // NB: IntelliJ will see the above @see statement as invalid, but it's correct: https://bugs.openjdk.java.net/browse/JDK-8031625 public static RiakClient newClient(RiakNode.Builder nodeBuilder, List addresses) throws UnknownHostException { final RiakCluster cluster = new RiakCluster.Builder(nodeBuilder, addresses).build(); @@ -402,6 +404,44 @@ public RiakFuture executeAsync(RiakCommand command) return command.executeAsync(cluster); } + /** + * Execute a StreamableRiakCommand asynchronously, and stream the results back before + * the command {@link RiakFuture#isDone() is done}. + *

+ * Calling this method causes the client to execute the provided + * StreamableRiakCommand asynchronously. + * It will immediately return a RiakFuture that contains an + * immediately available result (via {@link RiakFuture#get()}) that + * data will be streamed to. + * The RiakFuture will also keep track of the overall operation's progress + * with the {@link RiakFuture#isDone}, etc methods. + *

+ *

+ * Because the consumer thread will poll for new results, it is advisable to check the + * consumer thread's interrupted status via + * {@link Thread#isInterrupted() Thread.currentThread().isInterrupted() }, as the result + * iterator will not propagate an InterruptedException, but it will set the Thread's + * interrupted flag. + *

+ * @param StreamableRiakCommand's immediate return type, available before the command/operation is complete. + * @param The RiakCommand's query info type. + * @param command The RiakCommand to execute. + * @param timeoutMS The polling timeout in milliseconds for each result chunk. + * If the timeout is reached it will try again, instead of blocking indefinitely. + * If the value is too small (less than the average chunk arrival time), the + * result iterator will essentially busy wait. + * If the timeout is too large (much greater than the average chunk arrival time), + * the result iterator can block the consuming thread from seeing the done() + * status until the timeout is reached. + * @return a RiakFuture for the operation + * @since 2.1.0 + * @see RiakFuture + */ + public RiakFuture executeAsyncStreaming(StreamableRiakCommand command, int timeoutMS) + { + return command.executeAsyncStreaming(cluster, timeoutMS); + } + /** * Shut down the client and the underlying RiakCluster. *

diff --git a/src/main/java/com/basho/riak/client/api/StreamableRiakCommand.java b/src/main/java/com/basho/riak/client/api/StreamableRiakCommand.java new file mode 100644 index 000000000..ac983b63b --- /dev/null +++ b/src/main/java/com/basho/riak/client/api/StreamableRiakCommand.java @@ -0,0 +1,133 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.api; + +import com.basho.riak.client.api.commands.ChunkedResponseIterator; +import com.basho.riak.client.api.commands.ImmediateCoreFutureAdapter; +import com.basho.riak.client.core.*; + +import java.util.Iterator; + +/* + * The base class for all Streamable Riak Commands. + * Allows the user to either use {@link RiakCommand#executeAsync} and return a "batch-mode" result + * that is only available after the command is complete, or + * use {@link StreamableRiakCommand#executeAsyncStreaming} and return a "immediate" or "stream-mode" result + * that data will flow into. + * @param The response type returned by "streaming mode" {@link executeAsyncStreaming} + * @param The response type returned by the "batch mode" @{link executeAsync} + * @param The query info type + * @author Alex Moore + * @author Sergey Galkin + * @since 2.0 + */ +public abstract class StreamableRiakCommand extends GenericRiakCommand +{ + public static abstract class StreamableRiakCommandWithSameInfo extends StreamableRiakCommand + { + @Override + protected I convertInfo(I coreInfo) { + return coreInfo; + } + } + + public static abstract class StreamableResponse implements Iterable + { + protected ChunkedResponseIterator chunkedResponseIterator; + + /** + * Constructor for streamable response + * @param chunkedResponseIterator + */ + protected StreamableResponse(ChunkedResponseIterator chunkedResponseIterator) { + this.chunkedResponseIterator = chunkedResponseIterator; + } + + /** + * Constructor for not streamable response. + */ + protected StreamableResponse() + { + } + + /** + * Whether the results are to be streamed back. + * If true, results will appear in this class's iterator. + * If false, they will appear in the original result collection. + * @return true if the results are to be streamed. + */ + public boolean isStreaming() + { + return chunkedResponseIterator != null; + } + + /** + * Get an iterator over the result data. + * + * If using the streaming API, this method will block + * and wait for more data if none is immediately available. + * It is also advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return an iterator over the result data. + */ + @Override + public Iterator iterator() { + if (isStreaming()) { + assert chunkedResponseIterator != null; + return chunkedResponseIterator; + } + + throw new UnsupportedOperationException("Iterating is only supported for streamable response"); + } + } + + protected abstract R createResponse(int timeout, StreamingRiakFuture coreFuture); + + protected abstract PBStreamingFutureOperation buildCoreOperation(boolean streamResults); + + @Override + protected final FutureOperation buildCoreOperation() { + return buildCoreOperation(false); + } + + protected final RiakFuture executeAsyncStreaming(RiakCluster cluster, int timeout) + { + final PBStreamingFutureOperation coreOperation = buildCoreOperation(true); + final StreamingRiakFuture coreFuture = cluster.execute(coreOperation); + + final R r = createResponse(timeout, coreFuture); + + final ImmediateCoreFutureAdapter future = new ImmediateCoreFutureAdapter(coreFuture, r) + { + @Override + protected R convertResponse(CoreR response) + { + return StreamableRiakCommand.this.convertResponse(coreOperation, response); + } + + @Override + protected I convertQueryInfo(CoreI coreQueryInfo) + { + return StreamableRiakCommand.this.convertInfo(coreQueryInfo); + } + }; + + coreFuture.addListener(future); + return future; + } +} diff --git a/src/main/java/com/basho/riak/client/api/commands/ChunkedResponseIterator.java b/src/main/java/com/basho/riak/client/api/commands/ChunkedResponseIterator.java new file mode 100644 index 000000000..59f20a4fc --- /dev/null +++ b/src/main/java/com/basho/riak/client/api/commands/ChunkedResponseIterator.java @@ -0,0 +1,214 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.api.commands; + +import com.basho.riak.client.core.StreamingRiakFuture; +import com.basho.riak.client.core.util.BinaryValue; + +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TransferQueue; +import java.util.function.Function; + +/** + * Transforms a stream of response chunks to a Iterable of response items. + * + * When iterating over this class's {@link Iterator} this class will lazily walk + * through the response chunks's iterators and convert the items. + * It will also wait for more response chunks if none are available. + * + * Since this class polls for new "streaming" data, it is advisable + * to check {@link Thread#isInterrupted()} while using this class's + * {@link Iterator} in environments where thread interrupts must be obeyed. + * + * @param The final converted type that this class exposes as part of its iterator. + * @param The type of the response chunks, contains an Iterable<{@link CoreT}> + * @param The raw response type, will get converted to {@link FinalT}. + * @author Alex Moore + * @author Sergey Galkin + * @since 2.1.0 + */ +public class ChunkedResponseIterator, CoreT> implements Iterator +{ + private final int timeout; + private volatile BinaryValue continuation = null; + private final StreamingRiakFuture coreFuture; + private final TransferQueue chunkQueue; + private final Function createNext; + private final Function> getNextIterator; + private final Function getContinuationFn; + + protected Iterator currentIterator = null; + + public ChunkedResponseIterator(StreamingRiakFuture coreFuture, + int pollTimeout, + Function createNextFn, + Function> getNextIteratorFn) + { + this(coreFuture, pollTimeout, createNextFn, getNextIteratorFn, (x) -> null); + } + + public ChunkedResponseIterator(StreamingRiakFuture coreFuture, + int pollTimeout, + Function createNextFn, + Function> getNextIteratorFn, + Function getContinuationFn) + { + this.timeout = pollTimeout; + this.coreFuture = coreFuture; + this.chunkQueue = coreFuture.getResultsQueue(); + this.createNext = createNextFn; + this.getNextIterator = getNextIteratorFn; + this.getContinuationFn = getContinuationFn; + + // to kick of initial loading + hasNext(); + } + + /** + * Returns {@code true} if the iteration has more elements. + * (In other words, returns {@code true} if {@link #next} would + * return an element rather than throwing an exception.) + * + * This method will block and wait for more data if none is immediately available. + * + * Riak Java Client Note: Since this class polls for + * new "streaming" data, it is advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return {@code true} if the iteration has more elements + */ + @Override + public boolean hasNext() + { + // Check & clear interrupted flag so we don't get an + // InterruptedException every time if the user + // doesn't clear it / deal with it. + boolean interrupted = Thread.interrupted(); + Boolean dataLoaded = null; + + try + { + while (!currentIteratorHasNext() && dataLoaded == null) + { + try + { + dataLoaded = tryLoadNextChunkIterator(); + } + catch (InterruptedException ex) + { + interrupted = true; + } + } + return currentIteratorHasNext(); + } + finally + { + if (interrupted) + { + // Reset interrupted flag if we came in with it + // or we were interrupted while waiting. + Thread.currentThread().interrupt(); + } + } + } + + private boolean currentIteratorHasNext() + { + return currentIterator != null && currentIterator.hasNext(); + } + + private boolean possibleChunksRemaining() + { + // Chunks may remain if : + // Core Operation Not Done OR items still in chunk Queue + return !coreFuture.isDone() || !chunkQueue.isEmpty(); + } + + /** + * Returns whether this response contains a continuation. + * Only run this once the operation is complete, otherwise it will return true as it's + * @return Whether this response has a continuation. + */ + public boolean hasContinuation() + { + return continuation != null || possibleChunksRemaining(); + } + + /** + * Returns the current value of the continuation. + * Only run this once the operation is complete, or else you will get a null value. + * @return The continuation value (if any). + */ + public BinaryValue getContinuation() + { + return continuation; + } + + /** + * Returns the next element in the iteration. + * This method will block and wait for more data if none is immediately available. + * + * Riak Java Client Note: Since this class polls for + * new "streaming" data, it is advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return the next element in the iteration + * @throws NoSuchElementException if the iteration has no more elements + */ + @Override + public FinalT next() + { + if (hasNext()) + { + return createNext.apply(currentIterator.next()); + } + + throw new NoSuchElementException(); + } + + private boolean tryLoadNextChunkIterator() throws InterruptedException + { + this.currentIterator = null; + boolean populatedChunkLoaded = false; + + while (!populatedChunkLoaded && possibleChunksRemaining()) + { + final ChunkT nextChunk = chunkQueue.poll(timeout, TimeUnit.MILLISECONDS); + + if (nextChunk != null) + { + this.currentIterator = getNextIterator.apply(nextChunk); + populatedChunkLoaded = currentIteratorHasNext(); + + loadContinuation(nextChunk); + } + } + + return populatedChunkLoaded; + } + + private void loadContinuation(ChunkT nextChunk) + { + final BinaryValue fetchedContinuation = getContinuationFn.apply(nextChunk); + if (this.continuation == null && fetchedContinuation != null) + { + this.continuation = fetchedContinuation; + } + } +} diff --git a/src/main/java/com/basho/riak/client/api/commands/ImmediateCoreFutureAdapter.java b/src/main/java/com/basho/riak/client/api/commands/ImmediateCoreFutureAdapter.java new file mode 100644 index 000000000..b98c5b769 --- /dev/null +++ b/src/main/java/com/basho/riak/client/api/commands/ImmediateCoreFutureAdapter.java @@ -0,0 +1,83 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.api.commands; + +import com.basho.riak.client.core.RiakFuture; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Used when the converted response is available before the core future is complete. + * + * @param The core response type. + * @param The core query info type. + * @param The converted response type. + * @param The converted query info type. + * + * @author Alex Moore + * @since 2.1.0 + */ +public abstract class ImmediateCoreFutureAdapter extends CoreFutureAdapter +{ + private final T2 immediateResponse; + + protected ImmediateCoreFutureAdapter(RiakFuture coreFuture, T2 immediateResponse) + { + super(coreFuture); + this.immediateResponse = immediateResponse; + } + + @Override + public T2 get() throws InterruptedException, ExecutionException + { + return immediateResponse; + } + + @Override + public T2 get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException + { + return immediateResponse; + } + + @Override + public T2 getNow() + { + return immediateResponse; + } + + @Override + protected T2 convertResponse(T unused) { return null; } + + @Override + protected abstract S2 convertQueryInfo(S coreQueryInfo); + + public static abstract class SameQueryInfo extends ImmediateCoreFutureAdapter + { + protected SameQueryInfo(RiakFuture coreFuture, T2 immediateResponse) + { + super(coreFuture, immediateResponse); + } + + @Override + protected S convertQueryInfo(S coreQueryInfo) + { + return coreQueryInfo; + } + } +} diff --git a/src/main/java/com/basho/riak/client/api/commands/buckets/FetchBucketProperties.java b/src/main/java/com/basho/riak/client/api/commands/buckets/FetchBucketProperties.java index a4c393975..4722d14ac 100644 --- a/src/main/java/com/basho/riak/client/api/commands/buckets/FetchBucketProperties.java +++ b/src/main/java/com/basho/riak/client/api/commands/buckets/FetchBucketProperties.java @@ -16,10 +16,7 @@ package com.basho.riak.client.api.commands.buckets; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.FetchBucketPropsOperation; import com.basho.riak.client.core.query.Namespace; @@ -39,7 +36,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class FetchBucketProperties extends RiakCommand +public final class FetchBucketProperties extends AsIsRiakCommand { private final Namespace namespace; @@ -49,31 +46,7 @@ public FetchBucketProperties(Builder builder) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected FetchBucketPropsOperation.Response convertResponse(FetchBucketPropsOperation.Response coreResponse) - { - return coreResponse; - } - - @Override - protected Namespace convertQueryInfo(Namespace coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private FetchBucketPropsOperation buildCoreOperation() + protected FetchBucketPropsOperation buildCoreOperation() { return new FetchBucketPropsOperation.Builder(namespace).build(); } diff --git a/src/main/java/com/basho/riak/client/api/commands/buckets/ListBuckets.java b/src/main/java/com/basho/riak/client/api/commands/buckets/ListBuckets.java index a83bf4c07..80a33da31 100644 --- a/src/main/java/com/basho/riak/client/api/commands/buckets/ListBuckets.java +++ b/src/main/java/com/basho/riak/client/api/commands/buckets/ListBuckets.java @@ -15,11 +15,12 @@ */ package com.basho.riak.client.api.commands.buckets; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.commands.ChunkedResponseIterator; +import com.basho.riak.client.api.StreamableRiakCommand; +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.ListBucketsOperation; +import com.basho.riak.client.core.query.ConvertibleIterator; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; @@ -39,10 +40,30 @@ * System.out.println(ns.getBucketName()); * }} *

+ *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * ListBuckets lb = new ListBuckets.Builder("my_type").build();
+ * final RiakFuture streamFuture =
+ *     client.executeAsyncStreaming(lb, 200);
+ * final ListBuckets.StreamingResponse streamingResponse = streamFuture.get();
+ * for (Namespace ns : streamingResponse)
+ * {
+ *     System.out.println(ns.getBucketName());
+ * }}
+ *

* @author Dave Rusek + * @author Alex Moore * @since 2.0 */ -public final class ListBuckets extends RiakCommand +public final class ListBuckets extends StreamableRiakCommand.StreamableRiakCommandWithSameInfo { private final int timeout; private final BinaryValue type; @@ -54,31 +75,20 @@ public final class ListBuckets extends RiakCommand executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + ListBucketsOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(ListBucketsOperation.Response coreResponse) - { - return new Response(type, coreResponse.getBuckets()); - } + return new Response(type, coreResponse.getBuckets()); + } - @Override - protected BinaryValue convertQueryInfo(BinaryValue coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + @Override + protected Response createResponse(int timeout, StreamingRiakFuture coreFuture) + { + return new Response(type, timeout, coreFuture); } - private ListBucketsOperation buildCoreOperation() + @Override + protected ListBucketsOperation buildCoreOperation(boolean streamResults) { ListBucketsOperation.Builder builder = new ListBucketsOperation.Builder(); if (timeout > 0) @@ -91,6 +101,8 @@ private ListBucketsOperation buildCoreOperation() builder.withBucketType(type); } + builder.streamResults(streamResults); + return builder.build(); } @@ -108,52 +120,46 @@ private ListBucketsOperation buildCoreOperation() * *

*/ - public static class Response implements Iterable + public static class Response extends StreamableRiakCommand.StreamableResponse { private final BinaryValue type; private final List buckets; - public Response(BinaryValue type, List buckets) + Response(BinaryValue type, + int pollTimeout, + StreamingRiakFuture coreFuture) { - this.type = type; - this.buckets = buckets; - } - - @Override - public Iterator iterator() - { - return new Itr(buckets.iterator(), type); - } - } - - private static class Itr implements Iterator - { - private final Iterator iterator; - private final BinaryValue type; + super(new ChunkedResponseIterator<>(coreFuture, + pollTimeout, + (bucketName) -> new Namespace(type, bucketName), + (response) -> response.getBuckets().iterator())); - private Itr(Iterator iterator, BinaryValue type) - { - this.iterator = iterator; this.type = type; + this.buckets = null; } - @Override - public boolean hasNext() + public Response(BinaryValue type, List buckets) { - return iterator.hasNext(); + this.type = type; + this.buckets = buckets; } @Override - public Namespace next() + public Iterator iterator() { - BinaryValue bucket = iterator.next(); - return new Namespace(type, bucket); - } + if (isStreaming()) { + return super.iterator(); + } - @Override - public void remove() - { - iterator.remove(); + assert buckets != null; + return new ConvertibleIterator(buckets.iterator()) + { + @Override + protected Namespace convert(BinaryValue bucket) + { + return new Namespace(type, bucket); + } + }; } } diff --git a/src/main/java/com/basho/riak/client/api/commands/buckets/ResetBucketProperties.java b/src/main/java/com/basho/riak/client/api/commands/buckets/ResetBucketProperties.java index f952bab09..02cfea577 100644 --- a/src/main/java/com/basho/riak/client/api/commands/buckets/ResetBucketProperties.java +++ b/src/main/java/com/basho/riak/client/api/commands/buckets/ResetBucketProperties.java @@ -1,9 +1,6 @@ package com.basho.riak.client.api.commands.buckets; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ResetBucketPropsOperation; import com.basho.riak.client.core.query.Namespace; @@ -21,7 +18,7 @@ * @author Chris Mancini * @since 2.0 */ -public class ResetBucketProperties extends RiakCommand +public class ResetBucketProperties extends AsIsRiakCommand { private final Namespace namespace; @@ -31,32 +28,7 @@ public ResetBucketProperties(Builder builder) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected Namespace convertQueryInfo(Namespace coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private ResetBucketPropsOperation buildCoreOperation() - { + protected ResetBucketPropsOperation buildCoreOperation() { ResetBucketPropsOperation.Builder builder = new ResetBucketPropsOperation.Builder(namespace); diff --git a/src/main/java/com/basho/riak/client/api/commands/buckets/StoreBucketProperties.java b/src/main/java/com/basho/riak/client/api/commands/buckets/StoreBucketProperties.java index 14cd54d72..367678939 100644 --- a/src/main/java/com/basho/riak/client/api/commands/buckets/StoreBucketProperties.java +++ b/src/main/java/com/basho/riak/client/api/commands/buckets/StoreBucketProperties.java @@ -16,10 +16,7 @@ package com.basho.riak.client.api.commands.buckets; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.StoreBucketPropsOperation; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.functions.Function; @@ -39,7 +36,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class StoreBucketProperties extends RiakCommand +public final class StoreBucketProperties extends AsIsRiakCommand { private final Namespace namespace; private final Boolean allowMulti; @@ -95,31 +92,7 @@ public final class StoreBucketProperties extends RiakCommand } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected Namespace convertQueryInfo(Namespace coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private StoreBucketPropsOperation buildCoreOperation() + protected StoreBucketPropsOperation buildCoreOperation() { StoreBucketPropsOperation.Builder builder = new StoreBucketPropsOperation.Builder(namespace); diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchCounter.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchCounter.java index 815d80fac..7a47c3d05 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchCounter.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchCounter.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtFetchOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.crdt.types.RiakCounter; @@ -40,7 +38,7 @@ * @author Brian Roach * @since 2.0 */ -public final class FetchCounter extends FetchDatatype +public final class FetchCounter extends FetchDatatype { private FetchCounter(Builder builder) { @@ -48,38 +46,20 @@ private FetchCounter(Builder builder) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtFetchOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); + RiakDatatype element = coreResponse.getCrdtElement(); - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected FetchCounter.Response convertResponse(DtFetchOperation.Response coreResponse) - { - RiakDatatype element = coreResponse.getCrdtElement(); - - Context context = null; - if (coreResponse.hasContext()) - { - context = new Context(coreResponse.getContext()); - } - - RiakCounter datatype = extractDatatype(element); + Context context = null; + if (coreResponse.hasContext()) + { + context = new Context(coreResponse.getContext()); + } - return new Response(datatype, context); - } + RiakCounter datatype = extractDatatype(element); - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return new Response(datatype, context); } @Override diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchDatatype.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchDatatype.java index f6320346f..a78c9c492 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchDatatype.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchDatatype.java @@ -15,7 +15,7 @@ */ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.RiakCommand; +import com.basho.riak.client.api.GenericRiakCommand; import com.basho.riak.client.api.cap.Quorum; import com.basho.riak.client.api.commands.RiakOption; import com.basho.riak.client.core.operations.DtFetchOperation; @@ -31,7 +31,8 @@ * @author Dave Rusek * @since 2.0 */ -public abstract class FetchDatatype extends RiakCommand +public abstract class FetchDatatype extends + GenericRiakCommand.GenericRiakCommandWithSameInfo { private final Location location; private final Map, Object> options = new HashMap<>(); @@ -48,7 +49,7 @@ protected FetchDatatype(Builder builder) this.options.putAll(builder.options); } - public FetchDatatype withOption(Option option, V value) + public FetchDatatype withOption(Option option, V value) { options.put(option, value); return this; @@ -56,6 +57,7 @@ public FetchDatatype withOption(Option option, V value) public abstract T extractDatatype(RiakDatatype element); + @Override protected final DtFetchOperation buildCoreOperation() { DtFetchOperation.Builder builder = diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchHll.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchHll.java index aa84aad08..60c3dcdef 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchHll.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchHll.java @@ -1,5 +1,5 @@ /* - * Copyright 2016 Basho Technologies Inc + * Copyright 2016 Basho Technologies, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtFetchOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.crdt.types.RiakDatatype; @@ -38,9 +36,10 @@ * } * * @author Alex Moore - * @since 2.1 + * @since 2.1.0 */ -public final class FetchHll extends FetchDatatype + +public final class FetchHll extends FetchDatatype { private FetchHll(Builder builder) { @@ -48,32 +47,14 @@ private FetchHll(Builder builder) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) + protected RiakHll convertResponse(FutureOperation request, + DtFetchOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); + RiakDatatype element = coreResponse.getCrdtElement(); - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected RiakHll convertResponse(DtFetchOperation.Response coreResponse) - { - RiakDatatype element = coreResponse.getCrdtElement(); + RiakHll datatype = extractDatatype(element); - RiakHll datatype = extractDatatype(element); - - return datatype; - } - - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return datatype; } @Override diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchMap.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchMap.java index b5ca89daf..3f4101130 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchMap.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchMap.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtFetchOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.crdt.types.RiakDatatype; @@ -42,7 +40,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class FetchMap extends FetchDatatype +public final class FetchMap extends FetchDatatype { private FetchMap(Builder builder) { @@ -56,38 +54,20 @@ public RiakMap extractDatatype(RiakDatatype element) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtFetchOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); + RiakDatatype element = coreResponse.getCrdtElement(); - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected FetchMap.Response convertResponse(DtFetchOperation.Response coreResponse) - { - RiakDatatype element = coreResponse.getCrdtElement(); - - Context context = null; - if (coreResponse.hasContext()) - { - context = new Context(coreResponse.getContext()); - } - - RiakMap datatype = extractDatatype(element); + Context context = null; + if (coreResponse.hasContext()) + { + context = new Context(coreResponse.getContext()); + } - return new Response(datatype, context); - } + RiakMap datatype = extractDatatype(element); - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return new Response(datatype, context); } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchSet.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchSet.java index 3e2cf7e47..aed1ea64d 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchSet.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/FetchSet.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtFetchOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.crdt.types.RiakDatatype; @@ -41,7 +39,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class FetchSet extends FetchDatatype +public final class FetchSet extends FetchDatatype { private FetchSet(Builder builder) { @@ -49,38 +47,19 @@ private FetchSet(Builder builder) } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, DtFetchOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); + RiakDatatype element = coreResponse.getCrdtElement(); - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected FetchSet.Response convertResponse(DtFetchOperation.Response coreResponse) - { - RiakDatatype element = coreResponse.getCrdtElement(); - - Context context = null; - if (coreResponse.hasContext()) - { - context = new Context(coreResponse.getContext()); - } - - RiakSet datatype = extractDatatype(element); + Context context = null; + if (coreResponse.hasContext()) + { + context = new Context(coreResponse.getContext()); + } - return new Response(datatype, context); - } + RiakSet datatype = extractDatatype(element); - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return new Response(datatype, context); } @Override diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/HllUpdate.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/HllUpdate.java index 61cb5a642..ebb9c009a 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/HllUpdate.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/HllUpdate.java @@ -1,5 +1,5 @@ /* - * Copyright 2016 Basho Technologies Inc + * Copyright 2016 Basho Technologies, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.basho.riak.client.api.commands.datatypes; import com.basho.riak.client.core.query.crdt.ops.HllOp; @@ -30,7 +31,7 @@ * Riak HyperLogLog datatype. *

* @author Alex Moore - * @since 2.1 + * @since 2.1.0 */ public class HllUpdate implements DatatypeUpdate { diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateCounter.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateCounter.java index 38f794c71..19842e891 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateCounter.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateCounter.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtUpdateOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -49,55 +47,34 @@ * @author Brian Roach * @since 2.0 */ -public class UpdateCounter extends UpdateDatatype +public class UpdateCounter extends UpdateDatatype { - private final CounterUpdate update; - private UpdateCounter(Builder builder) { super(builder); - this.update = builder.update; } @Override - protected RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtUpdateOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation(update)); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(DtUpdateOperation.Response coreResponse) - { - RiakCounter counter = null; - if (coreResponse.hasCrdtElement()) - { - RiakDatatype element = coreResponse.getCrdtElement(); - counter = element.getAsCounter(); - } - BinaryValue returnedKey = coreResponse.hasGeneratedKey() - ? coreResponse.getGeneratedKey() - : null; - - Context returnedCtx = null; - if (coreResponse.hasContext()) - { - returnedCtx = new Context(coreResponse.getContext()); - } + RiakCounter counter = null; + if (coreResponse.hasCrdtElement()) + { + RiakDatatype element = coreResponse.getCrdtElement(); + counter = element.getAsCounter(); + } + BinaryValue returnedKey = coreResponse.hasGeneratedKey() + ? coreResponse.getGeneratedKey() + : null; - return new Response(returnedCtx, counter, returnedKey); - } + Context returnedCtx = null; + if (coreResponse.hasContext()) + { + returnedCtx = new Context(coreResponse.getContext()); + } - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return new Response(returnedCtx, counter, returnedKey); } /** @@ -116,8 +93,6 @@ private Response(Context context, RiakCounter datatype, BinaryValue generatedKey */ public static final class Builder extends UpdateDatatype.Builder { - private final CounterUpdate update; - /** * Construct a Builder for an UpdateCounter command. * @param location the location of the counter in Riak. @@ -125,12 +100,11 @@ public static final class Builder extends UpdateDatatype.Builder */ public Builder(Location location, CounterUpdate update) { - super(location); + super(location, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** @@ -146,12 +120,11 @@ public Builder(Location location, CounterUpdate update) */ public Builder(Namespace namespace, CounterUpdate update) { - super(namespace); + super(namespace, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateDatatype.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateDatatype.java index 8e7f05d25..2f3849299 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateDatatype.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateDatatype.java @@ -16,9 +16,10 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.RiakCommand; +import com.basho.riak.client.api.GenericRiakCommand; import com.basho.riak.client.api.cap.Quorum; import com.basho.riak.client.api.commands.RiakOption; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtUpdateOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -34,23 +35,27 @@ * @author Brian Roach * @since 2.0 */ -public abstract class UpdateDatatype extends RiakCommand +public abstract class UpdateDatatype + extends GenericRiakCommand.GenericRiakCommandWithSameInfo { protected final Namespace namespace; protected final BinaryValue key; private final Context ctx; private final Map, Object> options = new HashMap<>(); + private final DatatypeUpdate update; @SuppressWarnings("unchecked") UpdateDatatype(Builder builder) { this.namespace = builder.namespace; + this.update = builder.update; this.key = builder.key; this.ctx = builder.ctx; this.options.putAll(builder.options); } - protected final DtUpdateOperation buildCoreOperation(DatatypeUpdate update) + @Override + protected FutureOperation buildCoreOperation() { DtUpdateOperation.Builder builder; @@ -152,6 +157,7 @@ public Option(String name) */ public static abstract class Builder> { + private final DatatypeUpdate update; private final Namespace namespace; private BinaryValue key; private Context ctx; @@ -161,7 +167,7 @@ public static abstract class Builder> * Constructs a builder for a datatype update. * @param location the location of the datatype object in Riak. */ - Builder(Location location) + Builder(Location location, DatatypeUpdate update) { if (location == null) { @@ -169,6 +175,12 @@ public static abstract class Builder> } this.namespace = location.getNamespace(); this.key = location.getKey(); + + if (update == null) + { + throw new IllegalArgumentException("Update cannot be null."); + } + this.update = update; } /** @@ -181,13 +193,19 @@ public static abstract class Builder> * @param namespace the namespace to create the datatype. * @see Response#getGeneratedKey() */ - Builder(Namespace namespace) + Builder(Namespace namespace, DatatypeUpdate update) { if (namespace == null) { throw new IllegalArgumentException("Namespace cannot be null."); } this.namespace = namespace; + + if (update == null) + { + throw new IllegalArgumentException("Update cannot be null."); + } + this.update = update; } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateHll.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateHll.java index de78cd304..034e22f88 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateHll.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateHll.java @@ -1,5 +1,5 @@ /* - * Copyright 2016 Basho Technologies Inc + * Copyright 2016 Basho Technologies, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtUpdateOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -46,55 +44,35 @@ *

* * @author Alex Moore - * @since 2.1 + * @since 2.1.0 */ -public class UpdateHll extends UpdateDatatype + +public class UpdateHll extends UpdateDatatype { - private final HllUpdate update; - private UpdateHll(Builder builder) { super(builder); - this.update = builder.update; } @Override - protected RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtUpdateOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation(update)); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(DtUpdateOperation.Response coreResponse) - { - RiakHll hll = null; - if (coreResponse.hasCrdtElement()) - { - RiakDatatype element = coreResponse.getCrdtElement(); - hll = element.getAsHll(); - } - BinaryValue returnedKey = coreResponse.hasGeneratedKey() - ? coreResponse.getGeneratedKey() - : null; - Context returnedCtx = null; - if (coreResponse.hasContext()) - { - returnedCtx = new Context(coreResponse.getContext()); - } - return new Response(returnedCtx, hll, returnedKey); - } - - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + RiakHll hll = null; + if (coreResponse.hasCrdtElement()) + { + RiakDatatype element = coreResponse.getCrdtElement(); + hll = element.getAsHll(); + } + BinaryValue returnedKey = coreResponse.hasGeneratedKey() + ? coreResponse.getGeneratedKey() + : null; + Context returnedCtx = null; + if (coreResponse.hasContext()) + { + returnedCtx = new Context(coreResponse.getContext()); + } + return new Response(returnedCtx, hll, returnedKey); } /** @@ -102,8 +80,6 @@ protected Location convertQueryInfo(Location coreQueryInfo) */ public static class Builder extends UpdateDatatype.Builder { - private final HllUpdate update; - /** * Construct a Builder for an UpdateHll command. * @param location the location of the HyperLogLog in Riak. @@ -111,12 +87,11 @@ public static class Builder extends UpdateDatatype.Builder */ public Builder(Location location, HllUpdate update) { - super(location); + super(location, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** @@ -131,12 +106,11 @@ public Builder(Location location, HllUpdate update) */ public Builder(Namespace namespace, HllUpdate update) { - super(namespace); + super(namespace, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateMap.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateMap.java index d60f47bae..ba7786a40 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateMap.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateMap.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtUpdateOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -50,55 +48,34 @@ * @author Brian Roach * @since 2.0 */ -public class UpdateMap extends UpdateDatatype +public class UpdateMap extends UpdateDatatype { - private final MapUpdate update; - private UpdateMap(Builder builder) { super(builder); - this.update = builder.update; } @Override - protected RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtUpdateOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation(update)); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(DtUpdateOperation.Response coreResponse) - { - RiakMap map = null; - if (coreResponse.hasCrdtElement()) - { - RiakDatatype element = coreResponse.getCrdtElement(); - map = element.getAsMap(); - } - BinaryValue returnedKey = coreResponse.hasGeneratedKey() - ? coreResponse.getGeneratedKey() - : null; - - Context returnedCtx = null; - if (coreResponse.hasContext()) - { - returnedCtx = new Context(coreResponse.getContext()); - } + RiakMap map = null; + if (coreResponse.hasCrdtElement()) + { + RiakDatatype element = coreResponse.getCrdtElement(); + map = element.getAsMap(); + } + BinaryValue returnedKey = coreResponse.hasGeneratedKey() + ? coreResponse.getGeneratedKey() + : null; - return new Response(returnedCtx, map, returnedKey); - } + Context returnedCtx = null; + if (coreResponse.hasContext()) + { + returnedCtx = new Context(coreResponse.getContext()); + } - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return new Response(returnedCtx, map, returnedKey); } public static final class Response extends UpdateDatatype.Response @@ -114,8 +91,6 @@ private Response(Context context, RiakMap datatype, BinaryValue generatedKey) */ public static final class Builder extends UpdateDatatype.Builder { - private final MapUpdate update; - /** * Construct a Builder for an UpdateMap command. * @param location the location of the map in Riak. @@ -123,12 +98,11 @@ public static final class Builder extends UpdateDatatype.Builder */ public Builder(Location location, MapUpdate update) { - super(location); + super(location, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** @@ -144,12 +118,11 @@ public Builder(Location location, MapUpdate update) */ public Builder(Namespace namespace, MapUpdate update) { - super(namespace); + super(namespace, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } @Override diff --git a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateSet.java b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateSet.java index 062d49bc0..e5305d2f4 100644 --- a/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateSet.java +++ b/src/main/java/com/basho/riak/client/api/commands/datatypes/UpdateSet.java @@ -16,9 +16,7 @@ package com.basho.riak.client.api.commands.datatypes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.DtUpdateOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -49,53 +47,32 @@ * @author Brian Roach * @since 2.0 */ -public class UpdateSet extends UpdateDatatype +public class UpdateSet extends UpdateDatatype { - private final SetUpdate update; - private UpdateSet(Builder builder) { super(builder); - this.update = builder.update; } @Override - protected RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + DtUpdateOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation(update)); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(DtUpdateOperation.Response coreResponse) - { - RiakSet set = null; - if (coreResponse.hasCrdtElement()) - { - RiakDatatype element = coreResponse.getCrdtElement(); - set = element.getAsSet(); - } - BinaryValue returnedKey = coreResponse.hasGeneratedKey() - ? coreResponse.getGeneratedKey() - : null; - Context returnedCtx = null; - if (coreResponse.hasContext()) - { - returnedCtx = new Context(coreResponse.getContext()); - } - return new Response(returnedCtx, set, returnedKey); - } - - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + RiakSet set = null; + if (coreResponse.hasCrdtElement()) + { + RiakDatatype element = coreResponse.getCrdtElement(); + set = element.getAsSet(); + } + BinaryValue returnedKey = coreResponse.hasGeneratedKey() + ? coreResponse.getGeneratedKey() + : null; + Context returnedCtx = null; + if (coreResponse.hasContext()) + { + returnedCtx = new Context(coreResponse.getContext()); + } + return new Response(returnedCtx, set, returnedKey); } /** @@ -103,8 +80,6 @@ protected Location convertQueryInfo(Location coreQueryInfo) */ public static class Builder extends UpdateDatatype.Builder { - private final SetUpdate update; - /** * Construct a Builder for an UpdateSet command. * @param location the location of the set in Riak. @@ -112,12 +87,11 @@ public static class Builder extends UpdateDatatype.Builder */ public Builder(Location location, SetUpdate update) { - super(location); + super(location, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** @@ -133,12 +107,11 @@ public Builder(Location location, SetUpdate update) */ public Builder(Namespace namespace, SetUpdate update) { - super(namespace); + super(namespace, update); if (update == null) { throw new IllegalArgumentException("Update cannot be null"); } - this.update = update; } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/BigIntIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/BigIntIndexQuery.java index c01ba6ee1..382567ea2 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/BigIntIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/BigIntIndexQuery.java @@ -16,16 +16,12 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; + import java.math.BigInteger; -import java.util.ArrayList; -import java.util.List; /** * Performs a 2i query where the 2i index keys are numeric. @@ -42,10 +38,40 @@ * BigInteger key = new BigInteger(someReallyLongNumber); * BigIntIndexQuery q = new BigIntIndexQuery.Builder(ns, "my_index", key).build(); * BigIntIndexQuery.Response resp = client.execute(q);} + * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * BigInteger key = new BigInteger(someReallyLongNumber);
+ * BigIntIndexQuery q = new BigIntIndexQuery.Builder(ns, "my_index", key).build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * BigIntIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (BigIntIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKey().toString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Brian Roach + * @author Alex Moore + * @author Sergey Galkin * @since 2.0 */ -public class BigIntIndexQuery extends SecondaryIndexQuery +public class BigIntIndexQuery + extends SecondaryIndexQuery { private final IndexConverter converter; @@ -57,7 +83,7 @@ protected IndexConverter getConverter() protected BigIntIndexQuery(Init builder) { - super(builder); + super(builder, Response::new, Response::new); this.converter = new IndexConverter() { @Override @@ -79,42 +105,6 @@ public BinaryValue convert(BigInteger input) }; } - @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - executeCoreAsync(cluster); - - BigIntQueryFuture future = new BigIntQueryFuture(coreFuture); - coreFuture.addListener(future); - return future; - } - - protected final class BigIntQueryFuture - extends CoreFutureAdapter - { - public BigIntQueryFuture(RiakFuture coreFuture) - { - super(coreFuture); - } - - @Override - protected Response convertResponse(SecondaryIndexQueryOperation.Response coreResponse) - { - return new Response(namespace, coreResponse, converter); - } - - @Override - protected BigIntIndexQuery convertQueryInfo(SecondaryIndexQueryOperation.Query coreQueryInfo) - { - return BigIntIndexQuery.this; - } - } - protected static abstract class Init> extends SecondaryIndexQuery.Init { public Init(Namespace namespace, String indexName, S start, S end) @@ -206,34 +196,16 @@ public BigIntIndexQuery build() } } - public static class Response extends SecondaryIndexQuery.Response + public static class Response extends SecondaryIndexQuery.Response> { - protected Response(Namespace queryLocation, - SecondaryIndexQueryOperation.Response coreResponse, - IndexConverter converter) + Response(Namespace queryLocation, IndexConverter converter, int timeout, StreamingRiakFuture coreFuture) { - super(queryLocation, coreResponse, converter); + super(queryLocation, converter, timeout, coreFuture); } - @Override - public List getEntries() + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { - List convertedList = new ArrayList<>(); - for (SecondaryIndexQueryOperation.Response.Entry e : coreResponse.getEntryList()) - { - Location loc = getLocationFromCoreEntry(e); - Entry ce = new Entry(loc, e.getIndexKey(), converter); - convertedList.add(ce); - } - return convertedList; - } - - public class Entry extends SecondaryIndexQuery.Response.Entry - { - protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverter converter) - { - super(riakObjectLocation, indexKey, converter); - } + super(queryLocation, coreResponse, converter); } } } diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/BinIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/BinIndexQuery.java index c039bf003..dfb881948 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/BinIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/BinIndexQuery.java @@ -16,19 +16,14 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; -import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.indexes.IndexNames; import com.basho.riak.client.core.util.BinaryValue; import com.basho.riak.client.core.util.DefaultCharset; import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.List; /** * Performs a 2i query where the 2i index keys are strings. @@ -44,7 +39,35 @@ * BinIndexQuery q = new BinIndexQuery.Builder(ns, "my_index", key).build(); * BinIndexQuery.Response resp = client.execute(q);} * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * String key = "some_key";
+ * BinIndexQuery q = new BinIndexQuery.Builder(ns, "my_index", key).build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * BinIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (BinIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKey().toString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Brian Roach + * @author Alex Moore + * @author Sergey Galkin * @since 2.0 */ public class BinIndexQuery extends SecondaryIndexQuery @@ -54,7 +77,7 @@ public class BinIndexQuery extends SecondaryIndexQuery builder) { - super(builder); + super(builder, Response::new, Response::new); this.charset = builder.charset; this.converter = new StringIndexConverter(); } @@ -65,17 +88,6 @@ protected IndexConverter getConverter() return converter; } - @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - executeCoreAsync(cluster); - - BinQueryFuture future = new BinQueryFuture(coreFuture); - coreFuture.addListener(future); - return future; - } - @Override public boolean equals(Object o) { @@ -222,57 +234,16 @@ public BinIndexQuery build() } } - public static class Response extends SecondaryIndexQuery.Response + public static class Response extends SecondaryIndexQuery.Response> { - protected Response(Namespace queryLocation, - SecondaryIndexQueryOperation.Response coreResponse, - IndexConverter converter) + Response(Namespace queryLocation, IndexConverter converter, int timeout, StreamingRiakFuture coreFuture) { - super(queryLocation, coreResponse, converter); + super(queryLocation, converter, timeout, coreFuture); } - @Override - public List getEntries() + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { - List convertedList = new ArrayList<>(); - for (SecondaryIndexQueryOperation.Response.Entry e : coreResponse.getEntryList()) - { - Location loc = getLocationFromCoreEntry(e); - Entry ce = new Entry(loc, e.getIndexKey(), converter); - convertedList.add(ce); - } - return convertedList; - } - - public class Entry extends SecondaryIndexQuery.Response.Entry - { - protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverter converter) - { - super(riakObjectLocation, indexKey, converter); - } - } - } - - protected final class BinQueryFuture - extends CoreFutureAdapter - { - public BinQueryFuture(RiakFuture - coreFuture) - { - super(coreFuture); - } - - @Override - protected Response convertResponse(SecondaryIndexQueryOperation.Response coreResponse) - { - return new Response(namespace, coreResponse, converter); - } - - @Override - protected BinIndexQuery convertQueryInfo(SecondaryIndexQueryOperation.Query coreQueryInfo) - { - return BinIndexQuery.this; + super(queryLocation, coreResponse, converter); } } diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/BucketIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/BucketIndexQuery.java index 730209403..3ddce50ea 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/BucketIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/BucketIndexQuery.java @@ -1,9 +1,7 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.indexes.IndexNames; -import com.basho.riak.client.core.util.BinaryValue; /** * Performs a 2i query across the special $bucket index, for a known bucket, and returns the keys in that bucket. @@ -17,6 +15,31 @@ * BucketIndexQuery q = new BucketIndexQuery.Builder(ns).build(); * RawIndexquery.Response resp = client.execute(q);} * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * BucketIndexQuery q = new BucketIndexQuery.Builder(ns).build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * BinIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (BinIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKey().toString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Alex Moore * @since 2.0.7 */ diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/IntIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/IntIndexQuery.java index b28e93fe2..1b6eb9cee 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/IntIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/IntIndexQuery.java @@ -16,15 +16,10 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; -import java.util.ArrayList; -import java.util.List; /** * Performs a 2i query where the 2i index keys are numeric. @@ -39,7 +34,36 @@ * long key = 1234L; * IntIndexQuery q = new IntIndexQuery.Builder(ns, "my_index", key).build(); * IntIndexQuery.Response resp = client.execute(q);} + * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * long key = 1234L;
+ * IntIndexQuery q = new IntIndexQuery.Builder(ns, "my_index", key).build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * IntIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (IntIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKey().toString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Brian Roach + * @author Alex Moore + * @author Sergey Galkin * @since 2.0 */ public class IntIndexQuery extends SecondaryIndexQuery @@ -54,7 +78,7 @@ protected IndexConverter getConverter() protected IntIndexQuery(Init builder) { - super(builder); + super(builder, Response::new, Response::new); this.converter = new IndexConverter() { @Override @@ -77,41 +101,6 @@ public BinaryValue convert(Long input) }; } - @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - executeCoreAsync(cluster); - - IntQueryFuture future = new IntQueryFuture(coreFuture); - coreFuture.addListener(future); - return future; - } - - protected final class IntQueryFuture - extends CoreFutureAdapter - { - public IntQueryFuture(RiakFuture coreFuture) - { - super(coreFuture); - } - - @Override - protected Response convertResponse(SecondaryIndexQueryOperation.Response coreResponse) - { - return new Response(namespace, coreResponse, converter); - } - - @Override - protected IntIndexQuery convertQueryInfo(SecondaryIndexQueryOperation.Query coreQueryInfo) - { - return IntIndexQuery.this; - } - } - protected static abstract class Init> extends SecondaryIndexQuery.Init { public Init(Namespace namespace, String indexName, S start, S end) @@ -203,32 +192,16 @@ public IntIndexQuery build() } } - public static class Response extends SecondaryIndexQuery.Response + public static class Response extends SecondaryIndexQuery.Response> { - protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) - { - super(queryLocation, coreResponse, converter); - } - - @Override - public List getEntries() + Response(Namespace queryLocation, IndexConverter converter, int timeout, StreamingRiakFuture coreFuture) { - List convertedList = new ArrayList<>(); - for (SecondaryIndexQueryOperation.Response.Entry e : coreResponse.getEntryList()) - { - Location loc = getLocationFromCoreEntry(e); - Entry ce = new Entry(loc, e.getIndexKey(), converter); - convertedList.add(ce); - } - return convertedList; + super(queryLocation, converter, timeout, coreFuture); } - public class Entry extends SecondaryIndexQuery.Response.Entry + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { - protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverter converter) - { - super(riakObjectLocation, indexKey, converter); - } + super(queryLocation, coreResponse, converter); } } } diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/KeyIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/KeyIndexQuery.java index bfebfd24a..255c8bc67 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/KeyIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/KeyIndexQuery.java @@ -19,6 +19,31 @@ * KeyIndexQuery q = new KeyIndexQuery.Builder(ns, "foo10", "foo19").build(); * RawIndexquery.Response resp = client.execute(q);} * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * KeyIndexQuery q = new KeyIndexQuery.Builder(ns, "foo10", "foo19").build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * RawIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (BinIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKeyAsString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Alex Moore * @since 2.0.7 */ diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/RawIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/RawIndexQuery.java index c63a72972..54efce718 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/RawIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/RawIndexQuery.java @@ -16,15 +16,10 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; -import java.util.ArrayList; -import java.util.List; /** * Performs a 2i query where the 2i index keys are raw bytes. @@ -40,7 +35,37 @@ * Namespace ns = new Namespace("my_type", "my_bucket"); * RawIndexQuery q = new RawIndexQuery.Builder(ns, "my_index", Type._BIN, key).build(); * RawIndexquery.Response resp = client.execute(q);} + * + *

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * byte[] bytes = new byte[] { 1,2,3,4};
+ * BinaryValue key = BinaryValue.create(bytes);
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * RawIndexQuery q = new RawIndexQuery.Builder(ns, "my_index", Type._BIN, key).build();
+ * RiakFuture streamingFuture =
+ *     client.executeAsyncStreaming(q, 200);
+ * RawIndexQuery.StreamingResponse streamingResponse = streamingFuture.get();
+ *
+ * for (RawIndexQuery.Response.Entry e : streamingResponse)
+ * {
+ *     System.out.println(e.getRiakObjectLocation().getKey().toString());
+ * }
+ * // Wait for the command to fully finish.
+ * streamingFuture.await();
+ * // The StreamingResponse will also contain the continuation, if the operation returned one.
+ * streamingResponse.getContinuation(); }
+ *

+ * * @author Brian Roach + * @author Alex Moore + * @author Sergey Galkin * @since 2.0 */ public class RawIndexQuery extends SecondaryIndexQuery @@ -49,7 +74,7 @@ public class RawIndexQuery extends SecondaryIndexQuery builder) { - super(builder); + super(builder, Response::new, Response::new); this.converter = new IndexConverter() { @Override @@ -66,42 +91,6 @@ protected IndexConverter getConverter() return converter; } - @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - executeCoreAsync(cluster); - - RawQueryFuture future = new RawQueryFuture(coreFuture); - coreFuture.addListener(future); - return future; - } - - protected final class RawQueryFuture - extends CoreFutureAdapter - { - public RawQueryFuture(RiakFuture coreFuture) - { - super(coreFuture); - } - - @Override - protected Response convertResponse(SecondaryIndexQueryOperation.Response coreResponse) - { - return new Response(namespace, coreResponse, converter); - } - - @Override - protected RawIndexQuery convertQueryInfo(SecondaryIndexQueryOperation.Query coreQueryInfo) - { - return RawIndexQuery.this; - } - } - /** * Builder used to construct a RawIndexQuery command. */ @@ -156,34 +145,16 @@ public RawIndexQuery build() } } - public static class Response extends SecondaryIndexQuery.Response + public static class Response extends SecondaryIndexQuery.Response> { - protected Response(Namespace queryLocation, - SecondaryIndexQueryOperation.Response coreResponse, - IndexConverter converter) + Response(Namespace queryLocation, IndexConverter converter, int timeout, StreamingRiakFuture coreFuture) { - super(queryLocation, coreResponse, converter); + super(queryLocation, converter, timeout, coreFuture); } - @Override - public List getEntries() + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { - List convertedList = new ArrayList<>(); - for (SecondaryIndexQueryOperation.Response.Entry e : coreResponse.getEntryList()) - { - Location loc = getLocationFromCoreEntry(e); - Entry ce = new Entry(loc, e.getIndexKey(), converter); - convertedList.add(ce); - } - return convertedList; - } - - public class Entry extends SecondaryIndexQuery.Response.Entry - { - protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverter converter) - { - super(riakObjectLocation, indexKey, converter); - } + super(queryLocation, coreResponse, converter); } } } diff --git a/src/main/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQuery.java b/src/main/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQuery.java index f640725f5..52f6dba5b 100644 --- a/src/main/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQuery.java +++ b/src/main/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQuery.java @@ -16,14 +16,18 @@ package com.basho.riak.client.api.commands.indexes; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.StreamableRiakCommand; +import com.basho.riak.client.api.commands.ChunkedResponseIterator; +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; +import com.basho.riak.client.core.query.ConvertibleIterator; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; /** @@ -34,10 +38,30 @@ * * @param the type being used for the query. * @author Brian Roach + * @author Alex Moore + * @author Sergey Galkin * @since 2.0 */ -public abstract class SecondaryIndexQuery extends RiakCommand +public abstract class SecondaryIndexQuery, U extends SecondaryIndexQuery> + extends StreamableRiakCommand { + @FunctionalInterface + public interface StreamableResponseCreator> + { + R createResponse(Namespace queryLocation, + IndexConverter converter, + int timeout, + StreamingRiakFuture coreFuture); + } + + @FunctionalInterface + public interface GatherableResponseCreator> + { + R createResponse(Namespace queryLocation, + SecondaryIndexQueryOperation.Response coreResponse, + IndexConverter converter); + } + protected final Namespace namespace; protected final String indexName; protected final BinaryValue continuation; @@ -51,7 +75,11 @@ public abstract class SecondaryIndexQuery extends RiakCommand protected Integer timeout; protected final byte[] coverageContext; protected final boolean returnBody; - protected SecondaryIndexQuery(Init builder) + private final StreamableResponseCreator streamableResponseCreator; + private final GatherableResponseCreator gatherableResponseCreator; + + protected SecondaryIndexQuery(Init builder, StreamableResponseCreator streamableCreator, + GatherableResponseCreator gatherableResponseCreator) { this.namespace = builder.namespace; this.indexName = builder.indexName; @@ -66,6 +94,8 @@ protected SecondaryIndexQuery(Init builder) this.timeout = builder.timeout; this.coverageContext = builder.coverageContext; this.returnBody = builder.returnBody; + this.streamableResponseCreator = streamableCreator; + this.gatherableResponseCreator = gatherableResponseCreator; } protected abstract IndexConverter getConverter(); @@ -180,7 +210,8 @@ public Integer getTimeout() return timeout; } - protected final SecondaryIndexQueryOperation.Query createCoreQuery() + @Override + protected SecondaryIndexQueryOperation buildCoreOperation(boolean streamResults) { IndexConverter converter = getConverter(); @@ -203,7 +234,7 @@ protected final SecondaryIndexQueryOperation.Query createCoreQuery() else { coreQueryBuilder.withRangeStart(converter.convert(start)) - .withRangeEnd(converter.convert(end)); + .withRangeEnd(converter.convert(end)); } if (maxResults != null) @@ -220,16 +251,28 @@ protected final SecondaryIndexQueryOperation.Query createCoreQuery() { coreQueryBuilder.withCoverageContext(coverageContext); } - return coreQueryBuilder.build(); + + return new SecondaryIndexQueryOperation.Builder(coreQueryBuilder.build()).streamResults(streamResults).build(); + } + + @Override + protected S convertResponse(FutureOperation request, SecondaryIndexQueryOperation.Response coreResponse) + { + return gatherableResponseCreator.createResponse(namespace, coreResponse, getConverter()); } - protected RiakFuture executeCoreAsync(RiakCluster cluster) + @Override + @SuppressWarnings("unchecked") + protected U convertInfo(SecondaryIndexQueryOperation.Query coreInfo) { - SecondaryIndexQueryOperation.Builder builder = - new SecondaryIndexQueryOperation.Builder(this.createCoreQuery()); + return (U)SecondaryIndexQuery.this; + } - return cluster.execute(builder.build()); + @Override + protected S createResponse(int timeout, StreamingRiakFuture coreFuture) + { + return streamableResponseCreator.createResponse(namespace, getConverter(), timeout, coreFuture); } @Override @@ -531,7 +574,7 @@ public T withCoverageContext(byte[] coverageContext) * * It has protected access since, due to performance reasons, it might be used only for the Full Bucket Read * @param returnBody - * @return + * @return a reference to this object. */ protected T withReturnBody(boolean returnBody) { @@ -545,68 +588,172 @@ protected T withReturnBody(boolean returnBody) * * @param The type contained in the resposne. */ - public abstract static class Response + public static class Response> + extends StreamableResponse { final protected IndexConverter converter; final protected SecondaryIndexQueryOperation.Response coreResponse; final protected Namespace queryLocation; + protected Response(final Namespace queryLocation, IndexConverter converter, final int timeout, + final StreamingRiakFuture coreFuture) + { + this.queryLocation = queryLocation; + this.converter = converter; + this.coreResponse = null; + chunkedResponseIterator = new ChunkedResponseIterator( + coreFuture, timeout, null, + SecondaryIndexQueryOperation.Response::iterator, + SecondaryIndexQueryOperation.Response::getContinuation) + { + @SuppressWarnings("unchecked") + @Override + public E next() + { + final SecondaryIndexQueryOperation.Response.Entry coreEntity = currentIterator.next(); + return Response.this.createEntry(Response.this.queryLocation, coreEntity, converter); + } + }; + } + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { - this.coreResponse = coreResponse; - this.converter = converter; this.queryLocation = queryLocation; + this.converter = converter; + this.coreResponse = coreResponse; + } + + /** + * Get an iterator over the result data. + * + * If using the streaming API, this method will block + * and wait for more data if none is immediately available. + * It is also advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return an iterator over the result data. + */ + public Iterator iterator() + { + if (isStreaming()) { + return super.iterator(); + } + + return new ConvertibleIterator(coreResponse.getEntryList().iterator()) + { + @Override + protected E convert(SecondaryIndexQueryOperation.Response.Entry e) + { + return createEntry(queryLocation, e, converter); + } + }; } /** * Check if this response has a continuation. * + * If using the streaming API, this property's value + * may change while data is being received, therefore + * it is best to call it after the operation is complete. + * * @return true if the response contains a continuation. */ public boolean hasContinuation() { + if (isStreaming()) + { + return chunkedResponseIterator.hasContinuation(); + } + return coreResponse.hasContinuation(); } /** * Get the continuation from this response. * + * If using the streaming API, this property's value + * may change while data is being received, therefore + * it is best to call it after the operation is complete. + * * @return the continuation, or null if none is present. */ public BinaryValue getContinuation() { + if (isStreaming()) + { + return chunkedResponseIterator.getContinuation(); + } + return coreResponse.getContinuation(); } /** * Check is this response contains any entries. * + * If using the streaming API, this method will block + * and wait for more data if none is immediately available. + * It is also advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * * @return true if entries are present, false otherwise. */ public boolean hasEntries() { + if (isStreaming()) + { + return chunkedResponseIterator.hasNext(); + } + return !coreResponse.getEntryList().isEmpty(); } - protected final Location getLocationFromCoreEntry(SecondaryIndexQueryOperation.Response.Entry e) + /** + * Get a list of the result entries for this response. + * + * @return A list of result entries. + * @throws IllegalStateException when called while using the streaming API. + */ + public final List getEntries() + { + if(isStreaming()) + { + throw new IllegalStateException("Use the iterator() while using the streaming API"); + } + + final List coreEntries = coreResponse.getEntryList(); + final List convertedList = new ArrayList<>(coreEntries.size()); + + for (SecondaryIndexQueryOperation.Response.Entry e : coreEntries) + { + final E ce = createEntry(queryLocation, e, converter); + convertedList.add(ce); + } + return convertedList; + } + + @SuppressWarnings("unchecked") + protected E createEntry(Location location, SecondaryIndexQueryOperation.Response.Entry coreEntry, IndexConverter converter) { - Location loc = new Location(queryLocation, e.getObjectKey()); - return loc; + return (E)new Entry(location, coreEntry.getIndexKey(), converter); } - public abstract List getEntries(); + protected final E createEntry(Namespace namespace, SecondaryIndexQueryOperation.Response.Entry coreEntry, IndexConverter converter) + { + final Location loc = new Location(queryLocation, coreEntry.getObjectKey()); + return createEntry(loc, coreEntry, converter); + } - public abstract static class Entry + public static class Entry { - private final Location RiakObjectLocation; + private final Location riakObjectLocation; private final BinaryValue indexKey; private final IndexConverter converter; protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverter converter) { - this.RiakObjectLocation = riakObjectLocation; + this.riakObjectLocation = riakObjectLocation; this.indexKey = indexKey; this.converter = converter; } @@ -618,7 +765,7 @@ protected Entry(Location riakObjectLocation, BinaryValue indexKey, IndexConverte */ public Location getRiakObjectLocation() { - return RiakObjectLocation; + return riakObjectLocation; } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/CoveragePlan.java b/src/main/java/com/basho/riak/client/api/commands/kv/CoveragePlan.java index 877daaf1b..e2122955d 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/CoveragePlan.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/CoveragePlan.java @@ -15,10 +15,8 @@ */ package com.basho.riak.client.api.commands.kv; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.GenericRiakCommand; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.CoveragePlanOperation; import com.basho.riak.client.core.query.Namespace; @@ -27,7 +25,8 @@ * * @author Sergey Galkin */ -public class CoveragePlan extends RiakCommand +public class CoveragePlan extends GenericRiakCommand.GenericRiakCommandWithSameInfo { private final CoveragePlanOperation operation; @@ -37,27 +36,15 @@ private CoveragePlan(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - final RiakFuture coreFuture = cluster.execute(operation); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(CoveragePlanOperation.Response coreResponse) - { - return new Response(coreResponse); - } + protected CoveragePlanOperation buildCoreOperation() { + return operation; + } - @Override - protected Namespace convertQueryInfo(Namespace coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + @Override + protected Response convertResponse(FutureOperation request, + CoveragePlanOperation.Response coreResponse) + { + return new Response(coreResponse); } /** diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/DeleteValue.java b/src/main/java/com/basho/riak/client/api/commands/kv/DeleteValue.java index 2aa119cde..839ea4fa7 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/DeleteValue.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/DeleteValue.java @@ -15,13 +15,12 @@ */ package com.basho.riak.client.api.commands.kv; -import com.basho.riak.client.api.RiakCommand; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.api.cap.Quorum; import com.basho.riak.client.api.cap.VClock; import com.basho.riak.client.core.RiakCluster; import com.basho.riak.client.core.RiakFuture; import com.basho.riak.client.core.operations.DeleteOperation; -import com.basho.riak.client.api.commands.CoreFutureAdapter; import com.basho.riak.client.api.commands.RiakOption; import com.basho.riak.client.core.query.Location; @@ -64,7 +63,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class DeleteValue extends RiakCommand +public final class DeleteValue extends AsIsRiakCommand { private final Location location; private final Map, Object> options = new HashMap<>(); @@ -77,32 +76,18 @@ public DeleteValue(Builder builder) this.vClock = builder.vClock; } + /** + * To ensure that this method is accessible by {@link MultiDelete#executeBaseCommandAsync(DeleteValue, RiakCluster)} + * it MUST be overrode. + */ @Override protected RiakFuture executeAsync(RiakCluster cluster) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + return super.executeAsync(cluster); } - private DeleteOperation buildCoreOperation() + @Override + protected DeleteOperation buildCoreOperation() { DeleteOperation.Builder builder = new DeleteOperation.Builder(location); diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/FetchValue.java b/src/main/java/com/basho/riak/client/api/commands/kv/FetchValue.java index 7fb1c1d14..8e1876527 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/FetchValue.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/FetchValue.java @@ -15,13 +15,13 @@ */ package com.basho.riak.client.api.commands.kv; +import com.basho.riak.client.api.GenericRiakCommand; import com.basho.riak.client.api.cap.Quorum; import com.basho.riak.client.api.cap.VClock; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.RiakCluster; import com.basho.riak.client.core.operations.FetchOperation; -import com.basho.riak.client.api.RiakCommand; import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.api.commands.CoreFutureAdapter; import com.basho.riak.client.api.commands.RiakOption; import com.basho.riak.client.core.query.Location; @@ -70,7 +70,8 @@ * @since 2.0 * @see Response */ -public final class FetchValue extends RiakCommand +public final class FetchValue extends GenericRiakCommand.GenericRiakCommandWithSameInfo { private final Location location; private final Map, Object> options = new HashMap<>(); @@ -84,33 +85,22 @@ public final class FetchValue extends RiakCommand @Override protected final RiakFuture executeAsync(RiakCluster cluster) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(FetchOperation.Response coreResponse) - { - return new Response.Builder().withNotFound(coreResponse.isNotFound()) - .withUnchanged(coreResponse.isUnchanged()) - .withValues(coreResponse.getObjectList()) - .withLocation(location) // for ORM - .build(); - } + return super.executeAsync(cluster); + } - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + @Override + protected Response convertResponse(FutureOperation request, + FetchOperation.Response coreResponse) + { + return new Response.Builder().withNotFound(coreResponse.isNotFound()) + .withUnchanged(coreResponse.isUnchanged()) + .withValues(coreResponse.getObjectList()) + .withLocation(location) // for ORM + .build(); } - private FetchOperation buildCoreOperation() + @Override + protected FetchOperation buildCoreOperation() { FetchOperation.Builder builder = new FetchOperation.Builder(location); diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/FullBucketRead.java b/src/main/java/com/basho/riak/client/api/commands/kv/FullBucketRead.java index b349840c4..62856cd10 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/FullBucketRead.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/FullBucketRead.java @@ -15,10 +15,8 @@ */ package com.basho.riak.client.api.commands.kv; -import com.basho.riak.client.api.commands.CoreFutureAdapter; import com.basho.riak.client.api.commands.indexes.SecondaryIndexQuery; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.FetchOperation; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; import com.basho.riak.client.core.query.Location; @@ -26,7 +24,6 @@ import com.basho.riak.client.core.query.indexes.IndexNames; import com.basho.riak.client.core.util.BinaryValue; -import java.util.ArrayList; import java.util.List; /** @@ -46,7 +43,8 @@ * Note that this command must not be used without coverage context * for querying buckets that contain a big amount of data. * - * @author Sergey Galkin + * @author Sergey Galkin + * @author Alex Moore * @see CoveragePlan */ public class FullBucketRead extends SecondaryIndexQuery @@ -55,7 +53,7 @@ public class FullBucketRead extends SecondaryIndexQuery() { @Override @@ -72,42 +70,6 @@ protected IndexConverter getConverter() return converter; } - @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - executeCoreAsync(cluster); - - RawQueryFuture future = new RawQueryFuture(coreFuture); - coreFuture.addListener(future); - return future; - } - - protected final class RawQueryFuture - extends CoreFutureAdapter - { - public RawQueryFuture(RiakFuture coreFuture) - { - super(coreFuture); - } - - @Override - protected Response convertResponse(SecondaryIndexQueryOperation.Response coreResponse) - { - return new Response(namespace, coreResponse, converter); - } - - @Override - protected FullBucketRead convertQueryInfo(SecondaryIndexQueryOperation.Query coreQueryInfo) - { - return FullBucketRead.this; - } - } - /** * Builder used to construct a FullBucketRead command. */ @@ -235,58 +197,47 @@ public FullBucketRead build() } } - public static class Response extends SecondaryIndexQuery.Response + public static class Response extends SecondaryIndexQuery.Response { private transient List convertedList = null; - protected Response(Namespace queryLocation, - SecondaryIndexQueryOperation.Response coreResponse, - SecondaryIndexQuery.IndexConverter converter) + protected Response(Namespace queryLocation, IndexConverter converter, final int timeout, StreamingRiakFuture coreFuture) + { + super(queryLocation, converter, timeout, coreFuture); + } + + protected Response(Namespace queryLocation, SecondaryIndexQueryOperation.Response coreResponse, IndexConverter converter) { super(queryLocation, coreResponse, converter); } @Override - public List getEntries() + protected Entry createEntry(Location location, SecondaryIndexQueryOperation.Response.Entry coreEntry, IndexConverter converter) { - if (convertedList != null) + final FetchValue.Response fr; + if (coreEntry.hasBody()) { - return convertedList; + FetchOperation.Response resp = coreEntry.getBody(); + + // The following code has been copied from the FetchValue.executeAsync - CoreFutureAdapter + fr = new FetchValue.Response.Builder() + .withNotFound(resp.isNotFound()) + .withUnchanged(resp.isUnchanged()) + .withValues(resp.getObjectList()) + .withLocation(location) // for ORM + .build(); } - - convertedList = new ArrayList<>(coreResponse.getEntryList().size()); - for (SecondaryIndexQueryOperation.Response.Entry e : coreResponse.getEntryList()) + else { - final Location loc = getLocationFromCoreEntry(e); - - final FetchValue.Response fr; - if (e.hasBody()) - { - FetchOperation.Response resp = e.getBody(); - - // The following code has been copied from the FetchValue.executeAsync - CoreFutureAdapter - fr = new FetchValue.Response.Builder() - .withNotFound(resp.isNotFound()) - .withUnchanged(resp.isUnchanged()) - .withValues(resp.getObjectList()) - .withLocation(loc) // for ORM - .build(); - } - else - { - fr = null; - } - - Entry ce = new Entry(loc, fr); - convertedList.add(ce); + fr = null; } - return convertedList; + + return new Entry(location, fr); } - public static class Entry + public static class Entry extends SecondaryIndexQuery.Response.Entry { private final FetchValue.Response fetchedValue; - private final Location location; public Entry(Location location) { @@ -295,8 +246,8 @@ public Entry(Location location) public Entry(Location location, FetchValue.Response fetchedResponse) { + super(location, null, null); this.fetchedValue = fetchedResponse; - this.location = location; } public boolean hasFetchedValue() @@ -309,16 +260,11 @@ public FetchValue.Response getFetchedValue() return fetchedValue; } - public Location getLocation() - { - return location; - } - @Override public String toString() { return "FullBucketRead.Response.Entry{" + - "location=" + location + + "location=" + getRiakObjectLocation() + ", hasFetchedValue=" + hasFetchedValue() + '}'; } diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/KvBuilderBase.java b/src/main/java/com/basho/riak/client/api/commands/kv/KvBuilderBase.java index 1cbd4d099..0bb7a84d9 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/KvBuilderBase.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/KvBuilderBase.java @@ -27,7 +27,6 @@ protected KvBuilderBase(Location location) * * @param option the option * @param value the value for the option - * @return a reference to this object. */ protected void addOption(RiakOption option, Object value) { diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/ListKeys.java b/src/main/java/com/basho/riak/client/api/commands/kv/ListKeys.java index f0a12f511..beaf3bc71 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/ListKeys.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/ListKeys.java @@ -15,11 +15,12 @@ */ package com.basho.riak.client.api.commands.kv; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.StreamableRiakCommand; +import com.basho.riak.client.api.commands.ChunkedResponseIterator; +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.ListKeysOperation; -import com.basho.riak.client.api.commands.CoreFutureAdapter; +import com.basho.riak.client.core.query.ConvertibleIterator; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; @@ -44,13 +45,35 @@ * }} *

*

+ * You can also stream the results back before the operation is fully complete. + * This reduces the time between executing the operation and seeing a result, + * and reduces overall memory usage if the iterator is consumed quickly enough. + * The result iterable can only be iterated once though. + * If the thread is interrupted while the iterator is polling for more results, + * a {@link RuntimeException} will be thrown. + *

+ * {@code
+ * Namespace ns = new Namespace("my_type", "my_bucket");
+ * ListKeys lk = new ListKeys.Builder(ns).build();
+ * RiakFuture streamFuture =
+ *     client.executeAsyncStreaming(lk, 200);
+ * final ListKeys.StreamingResponse streamingResponse = streamFuture.get();
+ * ListKeys.Response response = client.execute(lk);
+ * for (Location l : streamingResponse)
+ * {
+ *     System.out.println(l.getKeyAsString());
+ * }}
+ *

+ *

* This is a very expensive operation and is not recommended for use on a production system *

* * @author Dave Rusek + * @author Alex Moore * @since 2.0 */ -public final class ListKeys extends RiakCommand +public final class ListKeys extends StreamableRiakCommand.StreamableRiakCommandWithSameInfo { private final Namespace namespace; private final int timeout; @@ -62,31 +85,20 @@ public final class ListKeys extends RiakCommand } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) + protected Response convertResponse(FutureOperation request, + ListKeysOperation.Response coreResponse) { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(ListKeysOperation.Response coreResponse) - { - return new Response(namespace, coreResponse.getKeys()); - } + return new Response(namespace, coreResponse.getKeys()); + } - @Override - protected Namespace convertQueryInfo(Namespace coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; + @Override + protected Response createResponse(int timeout, StreamingRiakFuture coreFuture) + { + return new Response(namespace, timeout, coreFuture); } - private ListKeysOperation buildCoreOperation() + @Override + protected ListKeysOperation buildCoreOperation(boolean streamResults) { ListKeysOperation.Builder builder = new ListKeysOperation.Builder(namespace); @@ -95,10 +107,12 @@ private ListKeysOperation buildCoreOperation() builder.withTimeout(timeout); } + builder.streamResults(streamResults); + return builder.build(); } - public static class Response implements Iterable + public static class Response extends StreamableRiakCommand.StreamableResponse { private final Namespace namespace; private final List keys; @@ -109,41 +123,36 @@ public Response(Namespace namespace, List keys) this.keys = keys; } - @Override - public Iterator iterator() + Response(Namespace namespace, + int pollTimeout, + StreamingRiakFuture coreFuture) { - return new Itr(namespace, keys.iterator()); - } - } - - private static class Itr implements Iterator - { - private final Iterator iterator; - private final Namespace namespace; + super(new ChunkedResponseIterator<>(coreFuture, + pollTimeout, + (key) -> new Location(namespace, key), + (nextChunk) -> nextChunk.getKeys().iterator())); - private Itr(Namespace namespace, Iterator iterator) - { - this.iterator = iterator; this.namespace = namespace; + this.keys = null; } @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public Location next() + public Iterator iterator() { - BinaryValue key = iterator.next(); - return new Location(namespace, key); - } + if (isStreaming()) + { + return super.iterator(); + } - @Override - public void remove() - { - iterator.remove(); + assert keys != null; + return new ConvertibleIterator(keys.iterator()) + { + @Override + protected Location convert(BinaryValue key) + { + return new Location(namespace, key); + } + }; } } diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/MultiDelete.java b/src/main/java/com/basho/riak/client/api/commands/kv/MultiDelete.java index d9690b86e..83f525c97 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/MultiDelete.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/MultiDelete.java @@ -21,7 +21,7 @@ *
  * {@code
  * MultiDelete multiDelete = ...;
- * MultiDelete.Response response = client.execute(multiDelete);
+ * MultiDelete.Response response = client.execute(multiDelete);}
  * 

*

* The maximum number of concurrent requests defaults to 10. This can be changed diff --git a/src/main/java/com/basho/riak/client/api/commands/kv/StoreValue.java b/src/main/java/com/basho/riak/client/api/commands/kv/StoreValue.java index 660d83fd4..3b7e17739 100644 --- a/src/main/java/com/basho/riak/client/api/commands/kv/StoreValue.java +++ b/src/main/java/com/basho/riak/client/api/commands/kv/StoreValue.java @@ -15,16 +15,16 @@ */ package com.basho.riak.client.api.commands.kv; +import com.basho.riak.client.api.GenericRiakCommand; import com.basho.riak.client.api.cap.Quorum; import com.basho.riak.client.api.cap.VClock; import com.basho.riak.client.api.convert.Converter; import com.basho.riak.client.api.convert.Converter.OrmExtracted; import com.basho.riak.client.api.convert.ConverterFactory; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.RiakCluster; import com.basho.riak.client.core.operations.StoreOperation; -import com.basho.riak.client.api.RiakCommand; import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.api.commands.CoreFutureAdapter; import com.basho.riak.client.api.commands.RiakOption; import com.basho.riak.client.core.util.BinaryValue; @@ -69,7 +69,8 @@ * @author Dave Rusek * @since 2.0 */ -public final class StoreValue extends RiakCommand +public final class StoreValue extends GenericRiakCommand.GenericRiakCommandWithSameInfo { private final Namespace namespace; private final BinaryValue key; @@ -88,9 +89,32 @@ public final class StoreValue extends RiakCommand this.vclock = builder.vclock; } - @SuppressWarnings("unchecked") + @Override + protected Response convertResponse(FutureOperation request, + StoreOperation.Response coreResponse) + { + Location loc = request.getQueryInfo(); + if (coreResponse.hasGeneratedKey()) + { + loc = new Location(loc.getNamespace(), coreResponse.getGeneratedKey()); + } + + return new Response.Builder() + .withValues(coreResponse.getObjectList()) + .withGeneratedKey(loc.getKey()) + .withLocation(loc) // for ORM + .build(); + } + @Override protected RiakFuture executeAsync(RiakCluster cluster) + { + return super.executeAsync(cluster); + } + + @SuppressWarnings("unchecked") + @Override + protected StoreOperation buildCoreOperation() { Converter converter; @@ -112,43 +136,6 @@ protected RiakFuture executeAsync(RiakCluster cluster) orm.getRiakObject().setVClock(vclock); } - RiakFuture coreFuture = - cluster.execute(buildCoreOperation(orm)); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(StoreOperation.Response coreResponse) - { - Namespace ns = orm.getNamespace(); - BinaryValue key = orm.getKey(); - if (coreResponse.hasGeneratedKey()) - { - key = coreResponse.getGeneratedKey(); - } - - Location loc = new Location(ns, key); - - return new Response.Builder() - .withValues(coreResponse.getObjectList()) - .withGeneratedKey(coreResponse.getGeneratedKey()) - .withLocation(loc) // for ORM - .build(); - } - - @Override - protected Location convertQueryInfo(Location coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private StoreOperation buildCoreOperation(OrmExtracted orm) - { StoreOperation.Builder builder; if (orm.hasKey()) diff --git a/src/main/java/com/basho/riak/client/api/commands/mapreduce/MapReduce.java b/src/main/java/com/basho/riak/client/api/commands/mapreduce/MapReduce.java index 4e53d6a1d..ca13a51b2 100644 --- a/src/main/java/com/basho/riak/client/api/commands/mapreduce/MapReduce.java +++ b/src/main/java/com/basho/riak/client/api/commands/mapreduce/MapReduce.java @@ -13,12 +13,11 @@ */ package com.basho.riak.client.api.commands.mapreduce; -import com.basho.riak.client.api.RiakCommand; import com.basho.riak.client.api.RiakException; -import com.basho.riak.client.api.commands.CoreFutureAdapter; +import com.basho.riak.client.api.StreamableRiakCommand; import com.basho.riak.client.api.convert.ConversionException; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.MapReduceOperation; import com.basho.riak.client.core.query.functions.Function; import com.basho.riak.client.core.util.BinaryValue; @@ -33,10 +32,8 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.Collection; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; +import java.util.*; +import java.util.concurrent.TransferQueue; /** * Base abstract class for all MapReduce commands. @@ -46,7 +43,8 @@ * @author Dave Rusek * @since 2.0 */ -public abstract class MapReduce extends RiakCommand +public abstract class MapReduce extends StreamableRiakCommand.StreamableRiakCommandWithSameInfo { private final MapReduceSpec spec; @@ -57,7 +55,7 @@ protected MapReduce(MapReduceInput input, Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) + protected MapReduceOperation buildCoreOperation(boolean streamResults) { BinaryValue jobSpec; try @@ -70,29 +68,22 @@ protected RiakFuture executeAsync(RiakCluster cluster) throw new RuntimeException(e); } - MapReduceOperation operation = new MapReduceOperation.Builder(jobSpec).build(); - - final RiakFuture coreFuture = cluster.execute(operation); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Response convertResponse(MapReduceOperation.Response coreResponse) - { - return new Response(coreResponse.getResults()); - } - - @Override - protected BinaryValue convertQueryInfo(BinaryValue coreQueryInfo) - { - return coreQueryInfo; - } - }; + return new MapReduceOperation.Builder(jobSpec) + .streamResults(streamResults) + .build(); + } - coreFuture.addListener(future); + @Override + protected Response convertResponse(FutureOperation request, + MapReduceOperation.Response coreResponse) + { + return new Response(coreResponse.getResults()); + } - return future; + @Override + protected Response createResponse(int timeout, StreamingRiakFuture coreFuture) + { + return new Response(coreFuture, timeout); } /** @@ -345,13 +336,28 @@ public T withLinkPhase(String bucket, String tag) /** * Response from a MapReduce command. */ - public static class Response + public static class Response extends StreamableRiakCommand.StreamableResponse { private final Map results; + private final MapReduceResponseIterator responseIterator; + + Response(StreamingRiakFuture coreFuture, + int pollTimeout) + { + responseIterator = new MapReduceResponseIterator(coreFuture, pollTimeout); + results = null; + } public Response(Map results) { this.results = results; + responseIterator = null; + } + + @Override + public boolean isStreaming() + { + return responseIterator != null; } public boolean hasResultForPhase(int i) @@ -394,5 +400,114 @@ private ArrayNode flattenResults() } return flatArray; } + + @Override + public Iterator iterator() + { + if (isStreaming()) { + return responseIterator; + } + + // TODO: add support for not streamable responses + throw new UnsupportedOperationException("Iterating is only supported for streamable response."); + } + + private class MapReduceResponseIterator implements Iterator + { + final StreamingRiakFuture coreFuture; + final TransferQueue resultsQueue; + private final int pollTimeout; + + MapReduceResponseIterator(StreamingRiakFuture coreFuture, + int pollTimeout) + { + this.coreFuture = coreFuture; + this.resultsQueue = coreFuture.getResultsQueue(); + this.pollTimeout = pollTimeout; + } + + /** + * Returns {@code true} if the iteration has more elements. + * (In other words, returns {@code true} if {@link #next} would + * return an element rather than throwing an exception.) + * + * This method will block and wait for more data if none is immediately available. + * + * Riak Java Client Note: Since this class polls for + * new "streaming" data, it is advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return {@code true} if the iteration has more elements + */ + @Override + public boolean hasNext() + { + // Check & clear interrupted flag so we don't get an + // InterruptedException every time if the user + // doesn't clear it / deal with it. + boolean interrupted = Thread.interrupted(); + try + { + boolean foundEntry = false; + boolean interruptedLastLoop; + + do + { + interruptedLastLoop = false; + + try + { + foundEntry = peekWaitForNextQueueEntry(); + } + catch (InterruptedException e) + { + interrupted = true; + interruptedLastLoop = true; + } + } while (interruptedLastLoop); + + return foundEntry; + } + finally + { + if (interrupted) + { + // Reset interrupted flag if we came in with it + // or we were interrupted while waiting. + Thread.currentThread().interrupt(); + } + } + } + + private boolean peekWaitForNextQueueEntry() throws InterruptedException + { + while (resultsQueue.isEmpty() && !coreFuture.isDone()) + { + if (resultsQueue.isEmpty()) + { + Thread.sleep(pollTimeout); + } + } + return !resultsQueue.isEmpty(); + } + + /** + * Returns the next element in the iteration. + * This method will block and wait for more data if none is immediately available. + * + * Riak Java Client Note: Since this class polls for + * new "streaming" data, it is advisable to check {@link Thread#isInterrupted()} + * in environments where thread interrupts must be obeyed. + * + * @return the next element in the iteration + * @throws NoSuchElementException if the iteration has no more elements + */ + @Override + public Response next() + { + final MapReduceOperation.Response responseChunk = resultsQueue.remove(); + return new Response(responseChunk.getResults()); + } + } } } diff --git a/src/main/java/com/basho/riak/client/api/commands/search/DeleteIndex.java b/src/main/java/com/basho/riak/client/api/commands/search/DeleteIndex.java index fa81b8ede..28484ffa7 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/DeleteIndex.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/DeleteIndex.java @@ -1,9 +1,6 @@ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.YzDeleteIndexOperation; /** @@ -11,7 +8,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class DeleteIndex extends RiakCommand +public final class DeleteIndex extends AsIsRiakCommand { private final String index; @@ -21,32 +18,7 @@ public final class DeleteIndex extends RiakCommand } @Override - protected final RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected String convertQueryInfo(String coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private YzDeleteIndexOperation buildCoreOperation() - { + protected YzDeleteIndexOperation buildCoreOperation() { return new YzDeleteIndexOperation.Builder(index).build(); } diff --git a/src/main/java/com/basho/riak/client/api/commands/search/FetchIndex.java b/src/main/java/com/basho/riak/client/api/commands/search/FetchIndex.java index cb5482fdd..691dbd27e 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/FetchIndex.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/FetchIndex.java @@ -16,10 +16,7 @@ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.YzFetchIndexOperation; /** @@ -27,7 +24,7 @@ * @author Dave Rusek * @since 2.0 */ -public class FetchIndex extends RiakCommand +public class FetchIndex extends AsIsRiakCommand { private final String index; @@ -37,31 +34,7 @@ public class FetchIndex extends RiakCommand executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected YzFetchIndexOperation.Response convertResponse(YzFetchIndexOperation.Response coreResponse) - { - return coreResponse; - } - - @Override - protected String convertQueryInfo(String coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private YzFetchIndexOperation buildCoreOperation() + protected YzFetchIndexOperation buildCoreOperation() { return new YzFetchIndexOperation.Builder().withIndexName(index).build(); } diff --git a/src/main/java/com/basho/riak/client/api/commands/search/FetchSchema.java b/src/main/java/com/basho/riak/client/api/commands/search/FetchSchema.java index 0c224a6c1..a5ce1df54 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/FetchSchema.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/FetchSchema.java @@ -16,10 +16,7 @@ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.YzGetSchemaOperation; /** @@ -27,7 +24,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class FetchSchema extends RiakCommand +public final class FetchSchema extends AsIsRiakCommand { private final String schema; @@ -36,32 +33,7 @@ public final class FetchSchema extends RiakCommand executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected YzGetSchemaOperation.Response convertResponse(YzGetSchemaOperation.Response coreResponse) - { - return coreResponse; - } - - @Override - protected String convertQueryInfo(String coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private YzGetSchemaOperation buildCoreOperation() + protected YzGetSchemaOperation buildCoreOperation() { return new YzGetSchemaOperation.Builder(schema).build(); } diff --git a/src/main/java/com/basho/riak/client/api/commands/search/Search.java b/src/main/java/com/basho/riak/client/api/commands/search/Search.java index d515128da..62512fc75 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/Search.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/Search.java @@ -15,11 +15,8 @@ */ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.api.commands.RiakOption; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; import com.basho.riak.client.core.operations.SearchOperation; import com.basho.riak.client.core.util.BinaryValue; @@ -61,7 +58,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class Search extends RiakCommand +public final class Search extends AsIsRiakCommand { /** * Enum that encapsulates the possible settings for a search command's presort setting. @@ -103,31 +100,7 @@ public Search(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected SearchOperation.Response convertResponse(SearchOperation.Response coreResponse) - { - return coreResponse; - } - - @Override - protected BinaryValue convertQueryInfo(BinaryValue coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private SearchOperation buildCoreOperation() + protected SearchOperation buildCoreOperation() { SearchOperation.Builder builder = new SearchOperation.Builder(BinaryValue.create(index), query); diff --git a/src/main/java/com/basho/riak/client/api/commands/search/StoreIndex.java b/src/main/java/com/basho/riak/client/api/commands/search/StoreIndex.java index c3657cac0..04e668387 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/StoreIndex.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/StoreIndex.java @@ -1,9 +1,6 @@ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.YzPutIndexOperation; import com.basho.riak.client.core.query.search.YokozunaIndex; @@ -17,7 +14,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class StoreIndex extends RiakCommand +public final class StoreIndex extends AsIsRiakCommand { private final Builder cmdBuilder; @@ -27,30 +24,7 @@ public final class StoreIndex extends RiakCommand } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected YokozunaIndex convertQueryInfo(YokozunaIndex coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private YzPutIndexOperation buildCoreOperation() + protected YzPutIndexOperation buildCoreOperation() { final YzPutIndexOperation.Builder opBuilder = new YzPutIndexOperation.Builder(cmdBuilder.index); diff --git a/src/main/java/com/basho/riak/client/api/commands/search/StoreSchema.java b/src/main/java/com/basho/riak/client/api/commands/search/StoreSchema.java index bd66f9078..5ec4749a7 100644 --- a/src/main/java/com/basho/riak/client/api/commands/search/StoreSchema.java +++ b/src/main/java/com/basho/riak/client/api/commands/search/StoreSchema.java @@ -1,9 +1,6 @@ package com.basho.riak.client.api.commands.search; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.YzPutSchemaOperation; import com.basho.riak.client.core.query.search.YokozunaSchema; @@ -16,7 +13,7 @@ * @author Dave Rusek * @since 2.0 */ -public final class StoreSchema extends RiakCommand +public final class StoreSchema extends AsIsRiakCommand { private final YokozunaSchema schema; @@ -26,31 +23,7 @@ public final class StoreSchema extends RiakCommand } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected Void convertResponse(Void coreResponse) - { - return coreResponse; - } - - @Override - protected YokozunaSchema convertQueryInfo(YokozunaSchema coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private YzPutSchemaOperation buildCoreOperation() + protected YzPutSchemaOperation buildCoreOperation() { return new YzPutSchemaOperation.Builder(schema).build(); } diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/CoveragePlan.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/CoveragePlan.java index 49a3428e2..cd1dee016 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/CoveragePlan.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/CoveragePlan.java @@ -15,13 +15,12 @@ */ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.ts.CoveragePlanOperation; import com.basho.riak.client.core.query.timeseries.CoveragePlanResult; -public class CoveragePlan extends RiakCommand +public class CoveragePlan extends AsIsRiakCommand { private final CoveragePlanOperation operation; @@ -31,10 +30,8 @@ private CoveragePlan(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture future = cluster.execute(operation); - return future; + protected FutureOperation buildCoreOperation() { + return operation; } public static class Builder extends CoveragePlanOperation.AbstractBuilder diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/CreateTable.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/CreateTable.java index 2f4453a77..b3538c826 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/CreateTable.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/CreateTable.java @@ -15,9 +15,8 @@ */ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.ts.CreateTableOperation; import com.basho.riak.client.core.query.timeseries.TableDefinition; @@ -28,7 +27,7 @@ * @author Sergey Galkin * @since 2.0.6 */ -public class CreateTable extends RiakCommand +public class CreateTable extends AsIsRiakCommand { private final Builder builder; @@ -38,12 +37,8 @@ private CreateTable(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - final RiakFuture future = - cluster.execute(builder.buildOperation()); - - return future; + protected FutureOperation buildCoreOperation() { + return builder.buildOperation(); } public static class Builder extends CreateTableOperation.AbstractBuilder diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/Delete.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/Delete.java index 60e774080..f2e1df4db 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/Delete.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/Delete.java @@ -1,8 +1,7 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; +import com.basho.riak.client.core.FutureOperation; import com.basho.riak.client.core.operations.ts.DeleteOperation; import com.basho.riak.client.core.query.timeseries.Cell; @@ -14,7 +13,7 @@ * @author Sergey Galkin * @since 2.0.3 */ -public class Delete extends RiakCommand +public class Delete extends AsIsRiakCommand { private final Builder builder; @@ -24,16 +23,7 @@ private Delete(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture future = - cluster.execute(buildCoreOperation()); - - return future; - } - - private DeleteOperation buildCoreOperation() - { + protected FutureOperation buildCoreOperation() { final DeleteOperation.Builder opBuilder = new DeleteOperation.Builder(this.builder.tableName, builder.keyValues); diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/DescribeTable.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/DescribeTable.java index a7d524ad7..d539631bc 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/DescribeTable.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/DescribeTable.java @@ -1,8 +1,6 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ts.DescribeTableOperation; import com.basho.riak.client.core.query.timeseries.TableDefinition; @@ -13,7 +11,7 @@ * @author Alex Moore * @since 2.0.4 */ -public class DescribeTable extends RiakCommand +public class DescribeTable extends AsIsRiakCommand { private final String tableName; @@ -34,15 +32,7 @@ public DescribeTable(String tableName) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture future = - cluster.execute(buildCoreOperation()); - - return future; - } - - private DescribeTableOperation buildCoreOperation() + protected DescribeTableOperation buildCoreOperation() { return new DescribeTableOperation(this.tableName); } diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/Fetch.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/Fetch.java index b81778c3a..c54836165 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/Fetch.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/Fetch.java @@ -1,8 +1,6 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ts.FetchOperation; import com.basho.riak.client.core.query.timeseries.Cell; import com.basho.riak.client.core.query.timeseries.QueryResult; @@ -15,7 +13,7 @@ * @author Sergey Galkin * @since 2.0.3 */ -public class Fetch extends RiakCommand +public class Fetch extends AsIsRiakCommand { private final Builder builder; @@ -25,15 +23,7 @@ private Fetch(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture future = - cluster.execute(buildCoreOperation()); - - return future; - } - - private FetchOperation buildCoreOperation() + protected FetchOperation buildCoreOperation() { final FetchOperation.Builder opBuilder = new FetchOperation.Builder(this.builder.tableName, builder.keyValues); diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/ListKeys.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/ListKeys.java index aeebf6968..d1568b3de 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/ListKeys.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/ListKeys.java @@ -1,9 +1,6 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.api.commands.CoreFutureAdapter; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ts.ListKeysOperation; import com.basho.riak.client.core.query.timeseries.QueryResult; @@ -15,7 +12,7 @@ * @author Sergey Galkin * @since 2.0.3 */ -public class ListKeys extends RiakCommand +public class ListKeys extends AsIsRiakCommand { private final String tableName; private final int timeout; @@ -27,31 +24,7 @@ private ListKeys(ListKeys.Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture coreFuture = - cluster.execute(buildCoreOperation()); - - CoreFutureAdapter future = - new CoreFutureAdapter(coreFuture) - { - @Override - protected QueryResult convertResponse(QueryResult coreResponse) - { - return coreResponse; - } - - @Override - protected String convertQueryInfo(String coreQueryInfo) - { - return coreQueryInfo; - } - }; - coreFuture.addListener(future); - return future; - } - - private ListKeysOperation buildCoreOperation() + protected ListKeysOperation buildCoreOperation() { ListKeysOperation.Builder builder = new ListKeysOperation.Builder(tableName); diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/Query.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/Query.java index 74bd09013..9334c4c15 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/Query.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/Query.java @@ -1,13 +1,9 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ts.QueryOperation; -import com.basho.riak.client.core.operations.ts.QueryOperation.Builder; import com.basho.riak.client.core.query.timeseries.QueryResult; import com.basho.riak.client.core.util.BinaryValue; -import com.google.protobuf.ByteString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,7 +20,7 @@ * @author Sergey Galkin * @since 2.0.3 */ -public class Query extends RiakCommand +public class Query extends AsIsRiakCommand { private final Builder builder; @@ -34,12 +30,7 @@ private Query(Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - return cluster.execute(buildCoreOperation()); - } - - private QueryOperation buildCoreOperation() + protected QueryOperation buildCoreOperation() { return new QueryOperation.Builder(builder.queryText) .withCoverageContext(builder.coverageContext) diff --git a/src/main/java/com/basho/riak/client/api/commands/timeseries/Store.java b/src/main/java/com/basho/riak/client/api/commands/timeseries/Store.java index ea1f06935..d0e7ea57d 100644 --- a/src/main/java/com/basho/riak/client/api/commands/timeseries/Store.java +++ b/src/main/java/com/basho/riak/client/api/commands/timeseries/Store.java @@ -1,8 +1,6 @@ package com.basho.riak.client.api.commands.timeseries; -import com.basho.riak.client.api.RiakCommand; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.api.AsIsRiakCommand; import com.basho.riak.client.core.operations.ts.StoreOperation; import com.basho.riak.client.core.query.timeseries.Row; @@ -21,7 +19,7 @@ * @author Sergey Galkin * @since 2.0.3 */ -public class Store extends RiakCommand +public class Store extends AsIsRiakCommand { private final Builder builder; @@ -31,15 +29,7 @@ private Store (Builder builder) } @Override - protected RiakFuture executeAsync(RiakCluster cluster) - { - RiakFuture future = - cluster.execute(buildCoreOperation()); - - return future; - } - - private StoreOperation buildCoreOperation() + protected StoreOperation buildCoreOperation() { return new StoreOperation.Builder(builder.tableName) .withRows(builder.rows) diff --git a/src/main/java/com/basho/riak/client/core/FutureOperation.java b/src/main/java/com/basho/riak/client/core/FutureOperation.java index 3e0195aea..8756c70c9 100644 --- a/src/main/java/com/basho/riak/client/core/FutureOperation.java +++ b/src/main/java/com/basho/riak/client/core/FutureOperation.java @@ -91,7 +91,7 @@ private enum State private final CountDownLatch latch = new CountDownLatch(1); private volatile OperationRetrier retrier; private volatile int remainingTries = 1; - private volatile List rawResponse = new LinkedList<>(); + private volatile List rawResponses = new LinkedList<>(); private volatile Throwable exception; private volatile T converted; private volatile State state = State.CREATED; @@ -194,7 +194,9 @@ public synchronized final void setResponse(RiakMessage rawResponse) { stateCheck(State.CREATED, State.WRITTEN, State.RETRY); U decodedMessage = decode(rawResponse); - this.rawResponse.add(decodedMessage); + + processMessage(decodedMessage); + exception = null; if (done(decodedMessage)) { @@ -208,6 +210,16 @@ public synchronized final void setResponse(RiakMessage rawResponse) } } + protected void processMessage(U decodedMessage) + { + processBatchMessage(decodedMessage); + } + + protected void processBatchMessage(U decodedMessage) + { + this.rawResponses.add(decodedMessage); + } + public synchronized final void setComplete() { logger.debug("Setting Complete on future"); @@ -322,7 +334,7 @@ private void tryConvertResponse() throws ExecutionException { try { - converted = convert(rawResponse); + converted = convert(rawResponses); } catch (IllegalArgumentException ex) { @@ -358,7 +370,7 @@ public final T getNow() { if (null == converted) { - converted = convert(rawResponse); + converted = convert(rawResponses); } return converted; diff --git a/src/main/java/com/basho/riak/client/core/PBStreamingFutureOperation.java b/src/main/java/com/basho/riak/client/core/PBStreamingFutureOperation.java new file mode 100644 index 000000000..c8f9e7602 --- /dev/null +++ b/src/main/java/com/basho/riak/client/core/PBStreamingFutureOperation.java @@ -0,0 +1,73 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.core; + +import com.basho.riak.client.core.operations.PBFutureOperation; +import com.basho.riak.protobuf.RiakKvPB; +import com.google.protobuf.GeneratedMessage; + +import java.util.concurrent.LinkedTransferQueue; +import java.util.concurrent.TransferQueue; + +/** + * @author Alex Moore + * @param The type returned by the streaming and non-streaming operation versions + * @param The protocol type returned + * @param Query info type + * @since 2.1.0 + */ +public abstract class PBStreamingFutureOperation + extends PBFutureOperation + implements StreamingRiakFuture +{ + private final TransferQueue responseQueue; + private boolean streamResults; + + protected PBStreamingFutureOperation(final byte reqMessageCode, + final byte respMessageCode, + final GeneratedMessage.Builder reqBuilder, + com.google.protobuf.Parser respParser, + boolean streamResults) + { + super(reqMessageCode, respMessageCode, reqBuilder, respParser); + this.streamResults = streamResults; + this.responseQueue = streamResults ? new LinkedTransferQueue<>() : null; + } + + @Override + protected void processMessage(ResponseType decodedMessage) + { + if (!streamResults) + { + super.processMessage(decodedMessage); + return; + } + + final ReturnType r = processStreamingChunk(decodedMessage); + assert this.responseQueue != null; + final boolean chunkAdded = responseQueue.offer(r); + assert chunkAdded; + } + + abstract protected ReturnType processStreamingChunk(ResponseType rawResponseChunk); + + public final TransferQueue getResultsQueue() + { + assert this.responseQueue != null; + return this.responseQueue; + } +} diff --git a/src/main/java/com/basho/riak/client/core/RiakCluster.java b/src/main/java/com/basho/riak/client/core/RiakCluster.java index e49a84e83..3613a9939 100644 --- a/src/main/java/com/basho/riak/client/core/RiakCluster.java +++ b/src/main/java/com/basho/riak/client/core/RiakCluster.java @@ -220,6 +220,20 @@ public boolean isDone() } public RiakFuture execute(FutureOperation operation) + { + return executeFutureOperation(operation); + } + + public StreamingRiakFuture execute(PBStreamingFutureOperation operation) + { + execute((FutureOperation)operation); + // N.B. Currently the operation and future are one in the same, + // so we can just return the operation to accomplish our + // StreamingRiakFuture return type contract. + return operation; + } + + private RiakFuture executeFutureOperation(FutureOperation operation) { stateCheck(State.RUNNING, State.QUEUING); operation.setRetrier(this, executionAttempts); diff --git a/src/main/java/com/basho/riak/client/core/RiakNode.java b/src/main/java/com/basho/riak/client/core/RiakNode.java index 7a2e009c2..e40d3c7e2 100644 --- a/src/main/java/com/basho/riak/client/core/RiakNode.java +++ b/src/main/java/com/basho/riak/client/core/RiakNode.java @@ -1603,7 +1603,7 @@ public static List buildNodes(Builder builder, List remoteAddr * @param builder a configured builder, used for common properties among the nodes * * @return a list of constructed RiakNodes - * @@since 2.0.6 + * @since 2.0.6 */ public static List buildNodes(Collection remoteHosts, Builder builder) { @@ -1630,8 +1630,8 @@ public static List buildNodes(Collection remoteHosts, Bui * Build a set of RiakNodes. * The provided builder will be used to construct a set of RiakNodes using the supplied addresses. * - * @see #buildNodes(Builder, List) * @since 2.0.3 + * @see #buildNodes(RiakNode.Builder, List) */ public static List buildNodes(Builder builder, String... remoteAddresses) throws UnknownHostException diff --git a/src/main/java/com/basho/riak/client/core/StreamingRiakFuture.java b/src/main/java/com/basho/riak/client/core/StreamingRiakFuture.java new file mode 100644 index 000000000..85bd2f604 --- /dev/null +++ b/src/main/java/com/basho/riak/client/core/StreamingRiakFuture.java @@ -0,0 +1,38 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.core; + +import java.util.concurrent.TransferQueue; + +/** + * The result of an asynchronous streaming (chunked) Riak operation. + * + * @author Alex Moore + * @param the streaming and non-streaming (response) return type + * @param The query info type + * + * @since 2.1.0 + */ +public interface StreamingRiakFuture + extends RiakFuture +{ + /** + * An Queue that provides the stream of results as they return from Riak. + * @return An Queue. + */ + TransferQueue getResultsQueue(); +} diff --git a/src/main/java/com/basho/riak/client/core/operations/ListBucketsOperation.java b/src/main/java/com/basho/riak/client/core/operations/ListBucketsOperation.java index ef31851f3..e3b922fec 100644 --- a/src/main/java/com/basho/riak/client/core/operations/ListBucketsOperation.java +++ b/src/main/java/com/basho/riak/client/core/operations/ListBucketsOperation.java @@ -15,26 +15,31 @@ */ package com.basho.riak.client.core.operations; -import com.basho.riak.client.core.FutureOperation; -import com.basho.riak.client.core.RiakMessage; +import com.basho.riak.client.core.PBStreamingFutureOperation; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; -import com.basho.riak.protobuf.RiakMessageCodes; import com.basho.riak.protobuf.RiakKvPB; +import com.basho.riak.protobuf.RiakMessageCodes; import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; -public class ListBucketsOperation extends FutureOperation +public class ListBucketsOperation extends PBStreamingFutureOperation { - private final RiakKvPB.RpbListBucketsReq.Builder reqBuilder; private final BinaryValue bucketType; private ListBucketsOperation(Builder builder) { - this.reqBuilder = builder.reqBuilder; + super(RiakMessageCodes.MSG_ListBucketsReq, + RiakMessageCodes.MSG_ListBucketsResp, + builder.reqBuilder, + RiakKvPB.RpbListBucketsResp.PARSER, + builder.streamResults); + this.bucketType = builder.bucketType; } @@ -50,32 +55,28 @@ protected ListBucketsOperation.Response convert(List buckets = new ArrayList<>(rawResponse.size()); for (RiakKvPB.RpbListBucketsResp resp : rawResponse) { - for (ByteString bucket : resp.getBucketsList()) - { - buckets.add(BinaryValue.unsafeCreate(bucket.toByteArray())); - } + buckets.addAll(convertSingleResponse(resp)); } + return new Response(bucketType, buckets); } - @Override - protected RiakMessage createChannelMessage() + private List convertSingleResponse(RiakKvPB.RpbListBucketsResp resp) { - return new RiakMessage(RiakMessageCodes.MSG_ListBucketsReq, reqBuilder.build().toByteArray()); + List buckets = new ArrayList<>(resp.getBucketsCount()); + + for (ByteString bucket : resp.getBucketsList()) + { + buckets.add(BinaryValue.unsafeCreate(bucket.toByteArray())); + } + return buckets; } @Override - protected RiakKvPB.RpbListBucketsResp decode(RiakMessage rawMessage) + protected Response processStreamingChunk(RiakKvPB.RpbListBucketsResp rawResponseChunk) { - try - { - Operations.checkPBMessageType(rawMessage, RiakMessageCodes.MSG_ListBucketsResp); - return RiakKvPB.RpbListBucketsResp.parseFrom(rawMessage.getData()); - } - catch (InvalidProtocolBufferException e) - { - throw new IllegalArgumentException("Invalid message received", e); - } + final List buckets = convertSingleResponse(rawResponseChunk); + return new Response(bucketType, buckets); } @Override @@ -88,6 +89,7 @@ public static class Builder { private final RiakKvPB.RpbListBucketsReq.Builder reqBuilder = RiakKvPB.RpbListBucketsReq.newBuilder().setStream(true); + private boolean streamResults = false; private BinaryValue bucketType = BinaryValue.create(Namespace.DEFAULT_BUCKET_TYPE); /** @@ -98,6 +100,7 @@ public Builder() /** * Provide a timeout for this operation. + * * @param timeout value in milliseconds * @return a reference to this object */ @@ -112,11 +115,12 @@ public Builder withTimeout(int timeout) } /** - * Set the bucket type. - * If unset {@link Namespace#DEFAULT_BUCKET_TYPE} is used. - * @param bucketType the bucket type to use - * @return A reference to this object. - */ + * Set the bucket type. + * If unset {@link Namespace#DEFAULT_BUCKET_TYPE} is used. + * + * @param bucketType the bucket type to use + * @return A reference to this object. + */ public Builder withBucketType(BinaryValue bucketType) { if (null == bucketType || bucketType.length() == 0) @@ -128,13 +132,31 @@ public Builder withBucketType(BinaryValue bucketType) return this; } + /** + * Set the streamResults flag. + * + * If unset or false, the entire result set will be available through the {@link ListBucketsOperation#get()} + * method once the operation is complete. + * + * If set to true, results will be pushed to the queue available through the {@link ListBucketsOperation#getResultsQueue()} + * method as soon as they are available. + * + * @param streamResults whether to stream results to {@link ListBucketsOperation#get()}(false), or {@link ListBucketsOperation#getResultsQueue()}(true) + * @return A reference to this object. + */ + public Builder streamResults(boolean streamResults) + { + this.streamResults = streamResults; + return this; + } + public ListBucketsOperation build() { return new ListBucketsOperation(this); } } - public static class Response + public static class Response implements Iterable { private final BinaryValue bucketType; private final List buckets; @@ -154,5 +176,11 @@ public List getBuckets() { return buckets; } + + @Override + public Iterator iterator() + { + return getBuckets().iterator(); + } } } diff --git a/src/main/java/com/basho/riak/client/core/operations/ListKeysOperation.java b/src/main/java/com/basho/riak/client/core/operations/ListKeysOperation.java index 923d258cc..9a0619b68 100644 --- a/src/main/java/com/basho/riak/client/core/operations/ListKeysOperation.java +++ b/src/main/java/com/basho/riak/client/core/operations/ListKeysOperation.java @@ -15,29 +15,29 @@ */ package com.basho.riak.client.core.operations; -import com.basho.riak.client.core.FutureOperation; -import com.basho.riak.client.core.RiakMessage; +import com.basho.riak.client.core.PBStreamingFutureOperation; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.util.BinaryValue; -import com.basho.riak.protobuf.RiakMessageCodes; import com.basho.riak.protobuf.RiakKvPB; +import com.basho.riak.protobuf.RiakMessageCodes; import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; -public class ListKeysOperation extends FutureOperation +public class ListKeysOperation extends PBStreamingFutureOperation { - private final Logger logger = LoggerFactory.getLogger("ListKeysOperation"); private final Namespace namespace; - private final RiakKvPB.RpbListKeysReq.Builder reqBuilder; private ListKeysOperation(Builder builder) { - this.reqBuilder = builder.reqBuilder; + super(RiakMessageCodes.MSG_ListKeysReq, + RiakMessageCodes.MSG_ListKeysResp, + builder.reqBuilder, + RiakKvPB.RpbListKeysResp.PARSER, + builder.streamResults); + this.namespace = builder.namespace; } @@ -47,32 +47,21 @@ protected Response convert(List rawResponse) Response.Builder builder = new Response.Builder(); for (RiakKvPB.RpbListKeysResp resp : rawResponse) { - for (ByteString bucket : resp.getKeysList()) - { - builder.addKey(BinaryValue.unsafeCreate(bucket.toByteArray())); - } + builder.addKeys(convertSingleResponse(resp)); } return builder.build(); } - @Override - protected RiakMessage createChannelMessage() + private List convertSingleResponse(RiakKvPB.RpbListKeysResp resp) { - return new RiakMessage(RiakMessageCodes.MSG_ListKeysReq, reqBuilder.build().toByteArray()); - } + List keys = new ArrayList<>(resp.getKeysCount()); - @Override - protected RiakKvPB.RpbListKeysResp decode(RiakMessage rawMessage) - { - try - { - Operations.checkPBMessageType(rawMessage, RiakMessageCodes.MSG_ListKeysResp); - return RiakKvPB.RpbListKeysResp.parseFrom(rawMessage.getData()); - } - catch (InvalidProtocolBufferException e) + for (ByteString key : resp.getKeysList()) { - throw new IllegalArgumentException("Invalid message received", e); + keys.add(BinaryValue.unsafeCreate(key.toByteArray())); } + + return keys; } @Override @@ -87,11 +76,18 @@ public Namespace getQueryInfo() return namespace; } + @Override + protected Response processStreamingChunk(RiakKvPB.RpbListKeysResp rawResponseChunk) + { + return new Response.Builder().addKeys(convertSingleResponse(rawResponseChunk)).build(); + } + public static class Builder { private final RiakKvPB.RpbListKeysReq.Builder reqBuilder = RiakKvPB.RpbListKeysReq.newBuilder(); private final Namespace namespace; + private boolean streamResults; /** * Construct a builder for a ListKeysOperaiton. @@ -118,13 +114,31 @@ public Builder withTimeout(int timeout) return this; } + /** + * Set the streamResults flag. + * + * If unset or false, the entire result set will be available through the {@link ListKeysOperation#get()} + * method once the operation is complete. + * + * If set to true, results will be pushed to the queue available through the {@link ListKeysOperation#getResultsQueue()} + * method as soon as they are available. + * + * @param streamResults whether to stream results to {@link ListKeysOperation#get()}(false), or {@link ListKeysOperation#getResultsQueue()}(true) + * @return A reference to this object. + */ + public Builder streamResults(boolean streamResults) + { + this.streamResults = streamResults; + return this; + } + public ListKeysOperation build() { return new ListKeysOperation(this); } } - public static class Response + public static class Response implements Iterable { private final List keys; private Response(Builder builder) @@ -137,6 +151,12 @@ public List getKeys() return keys; } + @Override + public Iterator iterator() + { + return keys.iterator(); + } + static class Builder { private List keys = new ArrayList<>(); diff --git a/src/main/java/com/basho/riak/client/core/operations/MapReduceOperation.java b/src/main/java/com/basho/riak/client/core/operations/MapReduceOperation.java index da810298c..73652ba6c 100644 --- a/src/main/java/com/basho/riak/client/core/operations/MapReduceOperation.java +++ b/src/main/java/com/basho/riak/client/core/operations/MapReduceOperation.java @@ -15,8 +15,7 @@ */ package com.basho.riak.client.core.operations; -import com.basho.riak.client.core.FutureOperation; -import com.basho.riak.client.core.RiakMessage; +import com.basho.riak.client.core.PBStreamingFutureOperation; import com.basho.riak.client.core.util.BinaryValue; import com.basho.riak.protobuf.RiakMessageCodes; import com.basho.riak.protobuf.RiakKvPB; @@ -25,12 +24,12 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,92 +37,79 @@ * A Map/Reduce Operation on Riak. No error checking is done on the content type of the content itself * with the exception to making sure they are provided. */ -public class MapReduceOperation extends FutureOperation +public class MapReduceOperation extends PBStreamingFutureOperation { - private final RiakKvPB.RpbMapRedReq.Builder reqBuilder; private final BinaryValue mapReduce; private final Logger logger = LoggerFactory.getLogger(MapReduceOperation.class); + private final ObjectMapper objectMapper = new ObjectMapper(); + private final JsonNodeFactory jsonNodeFactory = JsonNodeFactory.instance; private MapReduceOperation(Builder builder) { - this.reqBuilder = builder.reqBuilder; + super(RiakMessageCodes.MSG_MapRedReq, + RiakMessageCodes.MSG_MapRedResp, + builder.reqBuilder, + RiakKvPB.RpbMapRedResp.PARSER, + builder.streamResults); this.mapReduce = builder.mapReduce; } @Override protected Response convert(List rawResponse) { - // Riak streams the result back. Each message from Riak contains a int - // that tells you what phase the result is from. The result from a phase - // can span multiple messages. Each result chunk is a JSON array. - - final JsonNodeFactory factory = JsonNodeFactory.instance; - final ObjectMapper mapper = new ObjectMapper(); final Map resultMap = new LinkedHashMap<>(); - int phase = 0; - for (RiakKvPB.RpbMapRedResp response : rawResponse) { - if (response.hasPhase()) - { - phase = response.getPhase(); - } - if (response.hasResponse()) - { - ArrayNode jsonArray; - if (resultMap.containsKey(phase)) - { - jsonArray = resultMap.get(phase); - } - else - { - jsonArray = factory.arrayNode(); - resultMap.put(phase, jsonArray); - } - - JsonNode responseJson; - try - { - responseJson = mapper.readTree(response.getResponse().toStringUtf8()); - } - catch (IOException ex) - { - logger.error("Mapreduce job returned non-JSON; {}",response.getResponse().toStringUtf8()); - throw new RuntimeException("Non-JSON response from MR job", ex); - } - - if (responseJson.isArray()) - { - jsonArray.addAll((ArrayNode)responseJson); - } - else - { - logger.error("Mapreduce job returned JSON that wasn't an array; {}", response.getResponse().toStringUtf8()); - } - } + convertSingleResponse(resultMap, response); } return new Response(resultMap); } - @Override - protected RiakMessage createChannelMessage() + private void convertSingleResponse(Map resultMap, + RiakKvPB.RpbMapRedResp response) { - RiakKvPB.RpbMapRedReq request = reqBuilder.build(); - return new RiakMessage(RiakMessageCodes.MSG_MapRedReq, request.toByteArray()); - } + // Riak streams the result back. Each message from Riak contains a int + // that tells you what phase the result is from. The result from a phase + // can span multiple messages. Each result chunk is a JSON array. - @Override - protected RiakKvPB.RpbMapRedResp decode(RiakMessage rawMessage) - { - Operations.checkPBMessageType(rawMessage, RiakMessageCodes.MSG_MapRedResp); - try + int phase = 0; + if (response.hasPhase()) { - return RiakKvPB.RpbMapRedResp.parseFrom(rawMessage.getData()); + phase = response.getPhase(); } - catch (InvalidProtocolBufferException e) + if (response.hasResponse()) { - throw new IllegalArgumentException(e); + ArrayNode jsonArray; + if (resultMap.containsKey(phase)) + { + jsonArray = resultMap.get(phase); + } + else + { + jsonArray = jsonNodeFactory.arrayNode(); + resultMap.put(phase, jsonArray); + } + + JsonNode responseJson; + try + { + responseJson = objectMapper.readTree(response.getResponse().toStringUtf8()); + } + catch (IOException ex) + { + logger.error("Mapreduce job returned non-JSON; {}",response.getResponse().toStringUtf8()); + throw new RuntimeException("Non-JSON response from MR job", ex); + } + + if (responseJson.isArray()) + { + jsonArray.addAll((ArrayNode)responseJson); + } + else + { + logger.error("Mapreduce job returned JSON that wasn't an array; {}", response.getResponse().toStringUtf8()); + } } } @@ -139,11 +125,21 @@ public BinaryValue getQueryInfo() return mapReduce; } + @Override + protected Response processStreamingChunk(RiakKvPB.RpbMapRedResp rawResponseChunk) + { + final Map resultMap = new LinkedHashMap<>(); + + convertSingleResponse(resultMap, rawResponseChunk); + return new Response(resultMap); + } + public static class Builder { private final RiakKvPB.RpbMapRedReq.Builder reqBuilder = RiakKvPB.RpbMapRedReq.newBuilder(); private final BinaryValue mapReduce; + private boolean streamResults; /** * Create a MapReduce operation builder with the given function. @@ -162,6 +158,24 @@ public Builder(BinaryValue mapReduce) this.mapReduce = mapReduce; } + /** + * Set the streamResults flag. + * + * If unset or false, the entire result set will be available through the {@link MapReduceOperation#get()} + * method once the operation is complete. + * + * If set to true, results will be pushed to the queue available through the {@link MapReduceOperation#getResultsQueue()} + * method as soon as they are available. + * + * @param streamResults whether to stream results to {@link MapReduceOperation#get()}(false), or {@link MapReduceOperation#getResultsQueue()}(true) + * @return A reference to this object. + */ + public Builder streamResults(boolean streamResults) + { + this.streamResults = streamResults; + return this; + } + public MapReduceOperation build() { return new MapReduceOperation(this); diff --git a/src/main/java/com/basho/riak/client/core/operations/SecondaryIndexQueryOperation.java b/src/main/java/com/basho/riak/client/core/operations/SecondaryIndexQueryOperation.java index 39097366a..11b6ea461 100644 --- a/src/main/java/com/basho/riak/client/core/operations/SecondaryIndexQueryOperation.java +++ b/src/main/java/com/basho/riak/client/core/operations/SecondaryIndexQueryOperation.java @@ -15,20 +15,20 @@ */ package com.basho.riak.client.core.operations; -import com.basho.riak.client.core.FutureOperation; +import com.basho.riak.client.core.PBStreamingFutureOperation; import com.basho.riak.client.core.RiakMessage; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.indexes.IndexNames; import com.basho.riak.client.core.util.BinaryValue; -import com.basho.riak.protobuf.RiakMessageCodes; import com.basho.riak.protobuf.RiakKvPB; +import com.basho.riak.protobuf.RiakMessageCodes; import com.basho.riak.protobuf.RiakPB.RpbPair; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; import java.util.List; /** @@ -38,17 +38,27 @@ * @since 2.0 */ public class SecondaryIndexQueryOperation - extends FutureOperation + extends PBStreamingFutureOperation { - private final static Logger logger = LoggerFactory.getLogger(SecondaryIndexQueryOperation.class); private final RiakKvPB.RpbIndexReq pbReq; private final Query query; private SecondaryIndexQueryOperation(Builder builder) { + // Decide if we should release results as they come in (stream), or gather them all until the operation is + // done (not stream). + super(RiakMessageCodes.MSG_IndexReq, + RiakMessageCodes.MSG_IndexResp, + builder.pbReqBuilder, + null, + builder.streamResults); + // Yo dawg, we don't ever not want to use streaming. builder.pbReqBuilder.setStream(true); this.query = builder.query; + + // TODO: get rid of pbReq usage by switching to use query insted this.pbReq = builder.pbReqBuilder.build(); } @@ -58,50 +68,76 @@ protected SecondaryIndexQueryOperation.Response convert(List rawResponse SecondaryIndexQueryOperation.Response.Builder responseBuilder = new SecondaryIndexQueryOperation.Response.Builder(); + final boolean isIndexBodyResp = rawResponse != null && + !rawResponse.isEmpty() && + objectIsIndexBodyResp(rawResponse.get(0)); + for (Object o : rawResponse) { - if (o instanceof RiakKvPB.RpbIndexBodyResp) - { - assert pbReq.getReturnBody(); - final RiakKvPB.RpbIndexBodyResp bodyResp = (RiakKvPB.RpbIndexBodyResp)o; - convertBodies(responseBuilder, bodyResp); + convertSingleResponse(responseBuilder, isIndexBodyResp, o); + } - if (bodyResp.hasContinuation()) - { - responseBuilder.withContinuation( - BinaryValue.unsafeCreate(bodyResp.getContinuation().toByteArray())); - } - continue; - } + return responseBuilder.build(); + } - final RiakKvPB.RpbIndexResp pbEntry = (RiakKvPB.RpbIndexResp) o; + private boolean objectIsIndexBodyResp(Object o) + { + return o instanceof RiakKvPB.RpbIndexBodyResp; + } - /** - * The 2i API is inconsistent on the Riak side. If it's not - * a range query, return_terms is ignored it only returns the - * list of object keys and you have to have - * preserved the index key if you want to return it to the user - * with the results. - * - * Also, the $key index queries just ignore return_terms altogether. - */ + private void convertSingleResponse(Response.Builder responseBuilder, boolean isIndexBodyResp, Object o) + { + if (isIndexBodyResp) + { + convertIndexBodyResp(responseBuilder, o); + } + else + { + convertIndexResp(responseBuilder, o); + } + } - if (pbReq.getReturnTerms() && !query.indexName.toString().equalsIgnoreCase(IndexNames.KEY)) - { - convertTerms(responseBuilder, pbEntry); - } - else - { - convertKeys(responseBuilder, pbEntry); - } + private void convertIndexBodyResp(Response.Builder responseBuilder, Object o) + { + assert pbReq.getReturnBody(); + final RiakKvPB.RpbIndexBodyResp bodyResp = (RiakKvPB.RpbIndexBodyResp)o; + convertBodies(responseBuilder, bodyResp); - if (pbEntry.hasContinuation()) - { - responseBuilder.withContinuation( - BinaryValue.unsafeCreate(pbEntry.getContinuation().toByteArray())); - } + if (bodyResp.hasContinuation()) + { + responseBuilder.withContinuation( + BinaryValue.unsafeCreate(bodyResp.getContinuation().toByteArray())); + } + } + + private void convertIndexResp(Response.Builder responseBuilder, Object o) + { + final RiakKvPB.RpbIndexResp pbEntry = (RiakKvPB.RpbIndexResp) o; + + /** + * The 2i API is inconsistent on the Riak side. If it's not + * a range query, return_terms is ignored it only returns the + * list of object keys and you have to have + * preserved the index key if you want to return it to the user + * with the results. + * + * Also, the $key index queries just ignore return_terms altogether. + */ + + if (pbReq.getReturnTerms() && !query.indexName.toString().equalsIgnoreCase(IndexNames.KEY)) + { + convertTerms(responseBuilder, pbEntry); + } + else + { + convertKeys(responseBuilder, pbEntry); + } + + if (pbEntry.hasContinuation()) + { + responseBuilder.withContinuation( + BinaryValue.unsafeCreate(pbEntry.getContinuation().toByteArray())); } - return responseBuilder.build(); } private static void convertKeys(SecondaryIndexQueryOperation.Response.Builder builder, @@ -146,12 +182,6 @@ private void convertTerms(SecondaryIndexQueryOperation.Response.Builder builder, } } - @Override - protected RiakMessage createChannelMessage() - { - return new RiakMessage(RiakMessageCodes.MSG_IndexReq, pbReq.toByteArray()); - } - @Override protected Object decode(RiakMessage rawMessage) { @@ -196,6 +226,32 @@ public Query getQueryInfo() return query; } + @Override + protected Response processStreamingChunk(Object rawResponseChunk) + { + SecondaryIndexQueryOperation.Response.Builder responseBuilder = + new SecondaryIndexQueryOperation.Response.Builder(); + + final boolean bodyResp = objectIsIndexBodyResp(rawResponseChunk); + + convertSingleResponse(responseBuilder, bodyResp, rawResponseChunk); + + final Response response = responseBuilder.build(); + + if (response.hasContinuation()) + { + // Return the continuation in the normal fashion as well + final RiakKvPB.RpbIndexResp continuationOnlyResponse = + RiakKvPB.RpbIndexResp.newBuilder().setContinuation( + ByteString.copyFrom(response.getContinuation().unsafeGetValue())) + .build(); + + processBatchMessage(continuationOnlyResponse); + } + + return response; + } + /** * Builder that constructs a QueryOperation. */ @@ -203,6 +259,7 @@ public static class Builder { private final RiakKvPB.RpbIndexReq.Builder pbReqBuilder = RiakKvPB.RpbIndexReq.newBuilder(); private final Query query; + private boolean streamResults = false; /** * Constructs a builder for a QueryOperation. @@ -287,6 +344,26 @@ else if (query.getRangeStart() != null) } } + /** + * Set the streamResults flag. + *

+ * If unset or false, the entire result set will be available through the {@link ListKeysOperation#get()} + * method once the operation is complete. + *

+ * If set to true, results will be pushed to the queue available through the + * {@link ListKeysOperation#getResultsQueue()} + * method as soon as they are available. + * + * @param streamResults whether to stream results to {@link ListKeysOperation#get()}(false), or + * {@link ListKeysOperation#getResultsQueue()}(true) + * @return A reference to this object. + */ + public Builder streamResults(boolean streamResults) + { + this.streamResults = streamResults; + return this; + } + /** * Construct a new QueryOperation. * @return a QueryOperation @@ -644,7 +721,7 @@ else if (termFilter != null && indexName.toStringUtf8().endsWith("_int")) } } - public static class Response + public static class Response implements Iterable { private final BinaryValue continuation; private final List entryList; @@ -670,6 +747,12 @@ public List getEntryList() return entryList; } + @Override + public Iterator iterator() + { + return getEntryList().iterator(); + } + public static class Entry { private final BinaryValue indexKey; @@ -741,6 +824,12 @@ Builder addEntry(Response.Entry entry) return this; } + Builder addAllEntries(Collection entries) + { + entryList.addAll(entries); + return this; + } + Response build() { return new Response(this); diff --git a/src/main/java/com/basho/riak/client/core/query/ConvertibleIterator.java b/src/main/java/com/basho/riak/client/core/query/ConvertibleIterator.java new file mode 100644 index 000000000..7a93c349b --- /dev/null +++ b/src/main/java/com/basho/riak/client/core/query/ConvertibleIterator.java @@ -0,0 +1,35 @@ +package com.basho.riak.client.core.query; + +import com.basho.riak.protobuf.RiakTsPB; + +import java.util.Iterator; + +/** + * @author Sergey Galkin + * @author Alex Moore + * @since 2.0.3 + */ +public abstract class ConvertibleIterator implements Iterator { + private final Iterator iterator; + + public ConvertibleIterator(Iterator iterator) { + this.iterator = iterator; + } + + abstract protected D convert(S source); + + @Override + public final boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public final D next() { + return convert(iterator.next()); + } + + @Override + public final void remove() { + throw new UnsupportedOperationException(); + } +} \ No newline at end of file diff --git a/src/main/java/com/basho/riak/client/core/query/crdt/types/RiakHll.java b/src/main/java/com/basho/riak/client/core/query/crdt/types/RiakHll.java index f47af3704..18638623e 100644 --- a/src/main/java/com/basho/riak/client/core/query/crdt/types/RiakHll.java +++ b/src/main/java/com/basho/riak/client/core/query/crdt/types/RiakHll.java @@ -1,5 +1,5 @@ /* - * Copyright 2016 Basho Technologies Inc + * Copyright 2016 Basho Technologies, Inc. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -13,6 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.basho.riak.client.core.query.crdt.types; /** @@ -22,7 +23,7 @@ *

* * @author Alex Moore - * @since 2.1 + * @since 2.1.0 */ public class RiakHll extends RiakDatatype { diff --git a/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterable.java b/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterable.java index 19bb07e42..0edd900a3 100644 --- a/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterable.java +++ b/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterable.java @@ -28,7 +28,7 @@ public ImmutableIterablePBCell(Iterable source) @Override public Iterator iterator() { - return ConvertibleIterator.iterateAsPbCell(this.source.iterator()); + return ConvertibleIteratorUtils.iterateAsPbCell(this.source.iterator()); } } @@ -42,7 +42,7 @@ public ImmutableIterablePBRow(Iterable source) @Override public Iterator iterator() { - return ConvertibleIterator.iterateAsPbRow(this.source.iterator()); + return ConvertibleIteratorUtils.iterateAsPbRow(this.source.iterator()); } } @@ -56,7 +56,7 @@ public ImmutableIterableRow(Iterable source) @Override public Iterator iterator() { - return ConvertibleIterator.iterateAsRow(this.source.iterator()); + return ConvertibleIteratorUtils.iterateAsRow(this.source.iterator()); } } @@ -70,7 +70,7 @@ public ImmutableIterableCell(Iterable source) @Override public Iterator iterator() { - return ConvertibleIterator.iterateAsCell(this.source.iterator()); + return ConvertibleIteratorUtils.iterateAsCell(this.source.iterator()); } } diff --git a/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterator.java b/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIteratorUtils.java similarity index 82% rename from src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterator.java rename to src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIteratorUtils.java index 70be94d38..df861d69b 100644 --- a/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIterator.java +++ b/src/main/java/com/basho/riak/client/core/query/timeseries/ConvertibleIteratorUtils.java @@ -1,5 +1,6 @@ package com.basho.riak.client.core.query.timeseries; +import com.basho.riak.client.core.query.ConvertibleIterator; import com.basho.riak.protobuf.RiakTsPB; import java.util.Iterator; @@ -9,35 +10,11 @@ * @author Alex Moore * @since 2.0.3 */ -public abstract class ConvertibleIterator implements Iterator +class ConvertibleIteratorUtils { - private static final RiakTsPB.TsCell NullTSCell = RiakTsPB.TsCell.newBuilder().build(); - private final Iterator iterator; - - public ConvertibleIterator(Iterator iterator) - { - this.iterator = iterator; - } - - abstract protected D convert(S source); - - @Override - public final boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public final D next() - { - return convert(iterator.next()); - } + private ConvertibleIteratorUtils(){} - @Override - public final void remove() - { - throw new UnsupportedOperationException(); - } + private static final RiakTsPB.TsCell NullTSCell = RiakTsPB.TsCell.newBuilder().build(); private static class ImmutablePBCellIterator extends ConvertibleIterator { diff --git a/src/main/java/com/basho/riak/client/core/query/timeseries/FullColumnDescription.java b/src/main/java/com/basho/riak/client/core/query/timeseries/FullColumnDescription.java index 00ce5edf5..b56b1f794 100644 --- a/src/main/java/com/basho/riak/client/core/query/timeseries/FullColumnDescription.java +++ b/src/main/java/com/basho/riak/client/core/query/timeseries/FullColumnDescription.java @@ -55,7 +55,7 @@ public FullColumnDescription(String name, * @param quantum The {@link Quantum} setting if this column * is used in partition key time quantization. * Use - * {@link FullColumnDescription#FullColumnDescription(String, ColumnType, boolean, Integer)} + * {@link #FullColumnDescription(String, ColumnDescription.ColumnType, boolean, Integer)} * if the quantum is not needed. * @throws IllegalArgumentException if the Column Name or Column Type are null or empty, * or if the quantum is set on a non-Timestamp column, @@ -108,7 +108,7 @@ public FullColumnDescription(String name, * @param quantum The {@link Quantum} setting if this column * is used in partition key time quantization. * Use - * {@link FullColumnDescription#FullColumnDescription(String, ColumnType, boolean, Integer, Integer)} + * {@link #FullColumnDescription(String, ColumnDescription.ColumnType, boolean, Integer, Integer)} * if the quantum is not needed. * @throws IllegalArgumentException if the Column Name or Column Type are null or empty, * or if the quantum is set on a non-Timestamp column, diff --git a/src/main/java/com/basho/riak/client/core/query/timeseries/QueryResult.java b/src/main/java/com/basho/riak/client/core/query/timeseries/QueryResult.java index eb5f09e96..9eea203bc 100644 --- a/src/main/java/com/basho/riak/client/core/query/timeseries/QueryResult.java +++ b/src/main/java/com/basho/riak/client/core/query/timeseries/QueryResult.java @@ -4,7 +4,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -82,7 +81,7 @@ public Iterator iterator() } else { - return ConvertibleIterator.iterateAsRow(this.pbRows.iterator()); + return ConvertibleIteratorUtils.iterateAsRow(this.pbRows.iterator()); } } diff --git a/src/main/java/com/basho/riak/client/core/query/timeseries/Row.java b/src/main/java/com/basho/riak/client/core/query/timeseries/Row.java index efd102050..340ae4aca 100644 --- a/src/main/java/com/basho/riak/client/core/query/timeseries/Row.java +++ b/src/main/java/com/basho/riak/client/core/query/timeseries/Row.java @@ -102,7 +102,7 @@ public Iterator iterator() } else // if (pbRow != null) { - return ConvertibleIterator.iterateAsCell(pbRow.getCellsList().iterator()); + return ConvertibleIteratorUtils.iterateAsCell(pbRow.getCellsList().iterator()); } } diff --git a/src/test/java/com/basho/riak/client/api/commands/buckets/ListBucketsTest.java b/src/test/java/com/basho/riak/client/api/commands/buckets/ListBucketsTest.java index 51d959b29..e37b123ec 100644 --- a/src/test/java/com/basho/riak/client/api/commands/buckets/ListBucketsTest.java +++ b/src/test/java/com/basho/riak/client/api/commands/buckets/ListBucketsTest.java @@ -1,9 +1,7 @@ package com.basho.riak.client.api.commands.buckets; import com.basho.riak.client.api.RiakClient; -import com.basho.riak.client.core.FutureOperation; -import com.basho.riak.client.core.RiakCluster; -import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.*; import com.basho.riak.client.core.operations.ListBucketsOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -23,6 +21,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -33,7 +32,7 @@ public class ListBucketsTest { @Mock RiakCluster mockCluster; - @Mock RiakFuture mockFuture; + @Mock StreamingRiakFuture mockFuture; @Mock ListBucketsOperation.Response mockResponse; RiakClient client; @@ -49,21 +48,22 @@ public void init() throws Exception when(mockFuture.isCancelled()).thenReturn(false); when(mockFuture.isDone()).thenReturn(true); when(mockFuture.isSuccess()).thenReturn(true); - when(mockCluster.execute(any(FutureOperation.class))).thenReturn(mockFuture); + doReturn(mockFuture).when(mockCluster).execute(any(FutureOperation.class)); client = new RiakClient(mockCluster); } + @SuppressWarnings("unchecked") private void testListBuckets(String bucketType) throws Exception { final BinaryValue type = BinaryValue.createFromUtf8(bucketType); ListBuckets.Builder list = new ListBuckets.Builder(type); client.execute(list.build()); - ArgumentCaptor captor = - ArgumentCaptor.forClass(ListBucketsOperation.class); + ArgumentCaptor captor = + ArgumentCaptor.forClass(FutureOperation.class); verify(mockCluster).execute(captor.capture()); - ListBucketsOperation operation = captor.getValue(); + ListBucketsOperation operation = (ListBucketsOperation)captor.getValue(); RiakKvPB.RpbListBucketsReq.Builder builder = (RiakKvPB.RpbListBucketsReq.Builder) Whitebox.getInternalState(operation, "reqBuilder"); diff --git a/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListBuckets.java b/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListBuckets.java index 0735501aa..5c00cf163 100644 --- a/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListBuckets.java +++ b/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListBuckets.java @@ -1,59 +1,97 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package com.basho.riak.client.api.commands.buckets.itest; import com.basho.riak.client.api.RiakClient; import com.basho.riak.client.api.commands.buckets.ListBuckets; import com.basho.riak.client.api.commands.kv.StoreValue; -import com.basho.riak.client.core.operations.itest.ITestAutoCleanupBase; +import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.operations.itest.ITestBase; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.RiakObject; import com.basho.riak.client.core.util.BinaryValue; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.util.Iterator; import java.util.concurrent.ExecutionException; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assume.assumeTrue; /** * @author empovit + * @author Alex Moore * @since 2.0.3 */ -public class ITestListBuckets extends ITestAutoCleanupBase +public class ITestListBuckets extends ITestBase { - private final RiakClient client = new RiakClient(cluster); + private static final RiakClient client = new RiakClient(cluster); + private static final String bucketName = "ITestListBuckets"; + private static final Namespace defaultNamespace = new Namespace(bucketName); + private static final Namespace typedNamespace = new Namespace(bucketType.toString(), bucketName); + + @BeforeClass + public static void setup() throws ExecutionException, InterruptedException + { + storeTestObject(defaultNamespace); + if (testBucketType) + { + storeTestObject(typedNamespace); + } + } + + @AfterClass + public static void cleanup() throws ExecutionException, InterruptedException + { + resetAndEmptyBucket(defaultNamespace); + if (testBucketType) + { + resetAndEmptyBucket(typedNamespace); + } + } @Test public void testListBucketsDefaultType() throws InterruptedException, ExecutionException { - testListBuckets(Namespace.DEFAULT_BUCKET_TYPE); + testListBuckets(defaultNamespace); } @Test public void testListBucketsTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testListBuckets(bucketType.toString()); + testListBuckets(typedNamespace); } - private void testListBuckets(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testListBucketsStreamingTestType() throws InterruptedException, ExecutionException { - // Empty buckets do not show up - final BinaryValue key = BinaryValue.unsafeCreate("temp_key".getBytes()); - - RiakObject value = new RiakObject().setValue(BinaryValue.create("{\"value\":\"value\"}")); - - // Since bucket type in response is populated from the command's context, - // need a way to make sure the type is indeed as expected - use bucket type for bucket name - Location location = new Location(new Namespace(bucketType, bucketType), key); - StoreValue storeCommand = new StoreValue.Builder(value).withLocation(location).build(); - - client.execute(storeCommand); - - final BinaryValue typeBinary = BinaryValue.createFromUtf8(bucketType); + assumeTrue(testBucketType); + testListBucketsStreaming(typedNamespace); + } - ListBuckets listBucketsCommand = new ListBuckets.Builder(typeBinary).build(); + private void testListBuckets(Namespace namespace) throws InterruptedException, ExecutionException + { + ListBuckets listBucketsCommand = new ListBuckets.Builder(namespace.getBucketType()).build(); final ListBuckets.Response listResponse = client.execute(listBucketsCommand); @@ -63,9 +101,54 @@ private void testListBuckets(String bucketType) throws InterruptedException, Exe while (!found && iterator.hasNext()) { - found = iterator.next().getBucketName().equals(typeBinary); + found = iterator.next().getBucketName().toString().equals(bucketName); } assertTrue(found); } + + private void testListBucketsStreaming(Namespace namespace) throws InterruptedException, ExecutionException + { + ListBuckets listBucketsCommand = new ListBuckets.Builder(namespace.getBucketType()).build(); + + final RiakFuture streamingFuture = + client.executeAsyncStreaming(listBucketsCommand, 500); + + final ListBuckets.Response streamResponse = streamingFuture.get(); + final Iterator iterator = streamResponse.iterator(); + + assumeTrue(iterator.hasNext()); + boolean found = false; + + for (Namespace ns : streamResponse) + { + if (!found) + { + found = ns.getBucketName().toString().equals(bucketName); + } + } + + streamingFuture.await(); // Wait for command to finish, even if we've found our data + assumeTrue(streamingFuture.isDone()); + + assertFalse(iterator.hasNext()); + assertEquals(namespace.getBucketType(), streamingFuture.getQueryInfo()); + + assertTrue(found); + } + + private static void storeTestObject(Namespace namespace) throws ExecutionException, InterruptedException + { + // Empty buckets do not show up + final BinaryValue key = BinaryValue.unsafeCreate("temp_key".getBytes()); + + RiakObject value = new RiakObject().setValue(BinaryValue.create("{\"value\":\"value\"}")); + + // Since bucket type in response is populated from the command's context, + // need a way to make sure the type is indeed as expected - use bucket type for bucket name + Location location = new Location(namespace, key); + StoreValue storeCommand = new StoreValue.Builder(value).withLocation(location).build(); + + client.execute(storeCommand); + } } diff --git a/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListKeys.java b/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListKeys.java new file mode 100644 index 000000000..dec479329 --- /dev/null +++ b/src/test/java/com/basho/riak/client/api/commands/buckets/itest/ITestListKeys.java @@ -0,0 +1,118 @@ +/* + * Copyright 2016 Basho Technologies, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.basho.riak.client.api.commands.buckets.itest; + +import com.basho.riak.client.api.RiakClient; +import com.basho.riak.client.api.commands.kv.ListKeys; +import com.basho.riak.client.core.RiakFuture; +import com.basho.riak.client.core.operations.StoreOperation; +import com.basho.riak.client.core.operations.itest.ITestBase; +import com.basho.riak.client.core.query.Location; +import com.basho.riak.client.core.query.Namespace; +import com.basho.riak.client.core.query.RiakObject; +import com.basho.riak.client.core.util.BinaryValue; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Random; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; + +/** + * + * @author Alex Moore + */ +public class ITestListKeys extends ITestBase +{ + private static final RiakClient client = new RiakClient(cluster); + private static final String bucketName = "ITestListBuckets" + new Random().nextLong(); + private static final Namespace typedNamespace = new Namespace(bucketType.toString(), bucketName); + + @BeforeClass + public static void setup() throws ExecutionException, InterruptedException + { + if (testBucketType) + { + storeTestObjects(typedNamespace); + } + } + + @AfterClass + public static void cleanup() throws ExecutionException, InterruptedException + { + if (testBucketType) + { + resetAndEmptyBucket(typedNamespace); + } + } + + @Test + public void testLargeStreamingListKeys() throws ExecutionException, InterruptedException + { + assumeTrue(testBucketType); + + ListKeys lk = new ListKeys.Builder(typedNamespace).build(); + + final RiakFuture streamFuture = + client.executeAsyncStreaming(lk, 200); + + final ListKeys.Response streamingResponse = streamFuture.get(); + + int count = 0; + boolean foundLastKey = false; + + for (Location location : streamingResponse) + { + count++; + + if (!foundLastKey) + { + foundLastKey = location.getKeyAsString().equals("9999"); + } + } + + streamFuture.await(); + assertTrue(foundLastKey); + assertTrue(streamFuture.isDone()); + assertEquals(10000, count); + } + + private static void storeTestObjects(Namespace namespace) throws InterruptedException + { + final String value = "{\"value\":\"value\"}"; + final RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); + + for (int i = 0; i < 10000; i++) + { + final String key = Integer.toString(i); + + final Location location = new Location(namespace, key); + final StoreOperation storeOp = + new StoreOperation.Builder(location) + .withContent(rObj) + .build(); + + final RiakFuture execute = cluster.execute(storeOp); + execute.await(); + assertTrue(execute.isSuccess()); + } + } +} diff --git a/src/test/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQueryTest.java b/src/test/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQueryTest.java index 29eb320ab..e1cf1e53e 100644 --- a/src/test/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQueryTest.java +++ b/src/test/java/com/basho/riak/client/api/commands/indexes/SecondaryIndexQueryTest.java @@ -26,6 +26,7 @@ /** * * @author Brian Roach + * @author Alex Moore */ public class SecondaryIndexQueryTest { @@ -43,7 +44,7 @@ public void intIndexQueryBuildsCorrectly() .withTimeout(Integer.MAX_VALUE) .build(); - SecondaryIndexQueryOperation.Query query = iiq.createCoreQuery(); + SecondaryIndexQueryOperation.Query query = iiq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_int", query.getIndexName().toString()); @@ -61,7 +62,7 @@ public void intIndexQueryBuildsCorrectly() .withPaginationSort(true) .build(); - query = iiq.createCoreQuery(); + query = iiq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_int", query.getIndexName().toString()); @@ -98,7 +99,7 @@ public void binIndexQueryBuildsCorrectly() .withRegexTermFilter("filter") .build(); - SecondaryIndexQueryOperation.Query query = biq.createCoreQuery(); + SecondaryIndexQueryOperation.Query query = biq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_bin", query.getIndexName().toString()); @@ -116,7 +117,7 @@ public void binIndexQueryBuildsCorrectly() .withRegexTermFilter("filter") .build(); - query = biq.createCoreQuery(); + query = biq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_bin", query.getIndexName().toString()); @@ -128,6 +129,7 @@ public void binIndexQueryBuildsCorrectly() assertEquals(true, query.isReturnKeyAndIndex()); } + @Test public void rawIndexQueryBuildsCorrectly() { Namespace ns = new Namespace("bucket_type", "bucket_name"); @@ -140,7 +142,7 @@ public void rawIndexQueryBuildsCorrectly() new RawIndexQuery.Builder(ns, "test_index", SecondaryIndexQuery.Type._INT, indexMatch) .build(); - SecondaryIndexQueryOperation.Query query = riq.createCoreQuery(); + SecondaryIndexQueryOperation.Query query = riq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_int", query.getIndexName().toString()); @@ -151,7 +153,7 @@ public void rawIndexQueryBuildsCorrectly() indexStart, indexEnd) .build(); - query = riq.createCoreQuery(); + query = riq.buildCoreOperation(false).getQueryInfo(); assertEquals(ns, query.getNamespace()); assertEquals("test_index_int", query.getIndexName().toString()); diff --git a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBigIntIndexQuery.java b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBigIntIndexQuery.java index 4c74bf65b..672d8ce34 100644 --- a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBigIntIndexQuery.java +++ b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBigIntIndexQuery.java @@ -20,27 +20,41 @@ import com.basho.riak.client.api.annotations.RiakBucketName; import com.basho.riak.client.api.annotations.RiakIndex; import com.basho.riak.client.api.annotations.RiakKey; -import com.basho.riak.client.api.annotations.RiakVClock; -import com.basho.riak.client.api.cap.VClock; -import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.core.operations.itest.ITestAutoCleanupBase; import com.basho.riak.client.api.commands.indexes.BigIntIndexQuery; import com.basho.riak.client.api.commands.kv.StoreValue; +import com.basho.riak.client.core.RiakFuture; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + import java.math.BigInteger; import java.util.concurrent.ExecutionException; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.junit.Assume; -import org.junit.Test; /** - * * @author Brian Roach + * @author Alex Moore */ -public class ITestBigIntIndexQuery extends ITestAutoCleanupBase +public class ITestBigIntIndexQuery extends ITestIndexBase { + private static final String OBJECT_KEY_BASE = "index_test_object_key"; + private static final String INDEX_NAME = "test_index"; + private static final BigInteger INDEX_ENTRY = new BigInteger("91234567890123456789012345678901234567890"); + private static final BigInteger INDEX_ENTRY2 = new BigInteger("91234567890123456789012345678901234567898"); + private static final RiakClient client = new RiakClient(cluster); + + @BeforeClass + public static void setupData() throws InterruptedException + { + createIndexedPojo(0, INDEX_ENTRY); + createIndexedPojo(1, INDEX_ENTRY); + createIndexedPojo(2, INDEX_ENTRY2); + } + @Test public void testMatchQuery() throws InterruptedException, ExecutionException { @@ -48,64 +62,103 @@ public void testMatchQuery() throws InterruptedException, ExecutionException RiakClient client = new RiakClient(cluster); - IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key"; - ip.bucketName = bucketName.toString(); - ip.indexKey = new BigInteger("91234567890123456789012345678901234567890"); - ip.value = "My Object Value!"; + Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); - StoreValue sv = new StoreValue.Builder(ip).build(); - RiakFuture svFuture = client.executeAsync(sv); + BigIntIndexQuery indexQuery = new BigIntIndexQuery.Builder(ns, INDEX_NAME, INDEX_ENTRY).withKeyAndIndex(true) + .build(); + BigIntIndexQuery.Response indexResponse = client.execute(indexQuery); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + assertTrue(indexResponse.hasEntries()); + assertEquals(2, indexResponse.getEntries().size()); - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = new BigInteger("91234567890123456789012345678901234567890"); - ip2.value = "My Object Value!"; + assertFirstObjectFound(indexResponse); + } - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); + @Test + public void testRangeQuery() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + BigIntIndexQuery indexQuery = new BigIntIndexQuery.Builder(namespace, + INDEX_NAME, + INDEX_ENTRY, + INDEX_ENTRY2).withKeyAndIndex(true).build(); - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); + BigIntIndexQuery.Response indexResponse = client.execute(indexQuery); + assertTrue(indexResponse.hasEntries()); + assertEquals(3, indexResponse.getEntries().size()); + + assertFirstObjectFound(indexResponse); + } + + @Test + public void testStreamingRangeQuery() throws ExecutionException, InterruptedException + { + Assume.assumeTrue(test2i); + + BigIntIndexQuery indexQuery = new BigIntIndexQuery.Builder(namespace, + INDEX_NAME, + INDEX_ENTRY, + INDEX_ENTRY2).withKeyAndIndex(true).build(); + + final RiakFuture streamingFuture = + client.executeAsyncStreaming(indexQuery, 200); + + final BigIntIndexQuery.Response streamingResponse = streamingFuture.get(); - BigIntIndexQuery iiq = - new BigIntIndexQuery.Builder(ns, "test_index", new BigInteger("91234567890123456789012345678901234567890")) - .withKeyAndIndex(true).build(); - BigIntIndexQuery.Response iResp = client.execute(iiq); + assertTrue(streamingResponse.hasEntries()); - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); + boolean caught = false; + try + { + streamingResponse.getEntries(); + } + catch (IllegalStateException unused) + { + caught = true; + } + assertTrue(caught); + final String expectedObjectKey = objectKey(1); boolean found = false; - for (BigIntIndexQuery.Response.Entry e : iResp.getEntries()) + int size = 0; + + for (BigIntIndexQuery.Response.Entry e : streamingResponse) { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key")) + size++; + if (e.getRiakObjectLocation().getKey().toString().equals(expectedObjectKey)) { found = true; - assertEquals(ip.indexKey, e.getIndexKey()); + assertEquals(INDEX_ENTRY, e.getIndexKey()); } } assertTrue(found); + assertEquals(3, size); } - @Test - public void testRangeQuery() throws InterruptedException, ExecutionException + private boolean assertFirstObjectFound(BigIntIndexQuery.Response response) { - Assume.assumeTrue(test2i); + final String expectedObjectKey = objectKey(1); + boolean found = false; - RiakClient client = new RiakClient(cluster); + for (BigIntIndexQuery.Response.Entry e : response.getEntries()) + { + if (e.getRiakObjectLocation().getKey().toString().equals(expectedObjectKey)) + { + found = true; + assertEquals(INDEX_ENTRY, e.getIndexKey()); + } + } + return found; + } + private static void createIndexedPojo(int keySuffix, BigInteger indexValue) throws InterruptedException + { IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key1"; - ip.bucketName = bucketName.toString(); - ip.indexKey = new BigInteger("91234567890123456789012345678901234567890"); + ip.key = objectKey(keySuffix); + ip.bucketName = namespace.getBucketNameAsString(); + ip.indexKey = indexValue; ip.value = "My Object Value!"; StoreValue sv = new StoreValue.Builder(ip).build(); @@ -113,45 +166,14 @@ public void testRangeQuery() throws InterruptedException, ExecutionException svFuture.await(); assertTrue(svFuture.isSuccess()); + } - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = new BigInteger("91234567890123456789012345678901234567898"); - ip2.value = "My Object Value!"; - - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); - - svFuture.await(); - assertTrue(svFuture.isSuccess()); - - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); - BigIntIndexQuery iiq = - new BigIntIndexQuery.Builder(ns, "test_index", - new BigInteger("91234567890123456789012345678901234567890"), - new BigInteger("91234567890123456789012345678901234567898")) - .withKeyAndIndex(true) - .build(); - - BigIntIndexQuery.Response iResp = client.execute(iiq); - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); - - boolean found = false; - for (BigIntIndexQuery.Response.Entry e : iResp.getEntries()) - { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key1")) - { - found = true; - assertEquals(ip.indexKey, e.getIndexKey()); - } - } - - assertTrue(found); + private static String objectKey(int suffix) + { + return OBJECT_KEY_BASE + suffix; } - public static class IndexedPojo + private static class IndexedPojo { @RiakKey public String key; @@ -159,12 +181,9 @@ public static class IndexedPojo @RiakBucketName public String bucketName; - @RiakIndex(name="test_index") + @RiakIndex(name = INDEX_NAME) BigInteger indexKey; - @RiakVClock - VClock vclock; - public String value; } } diff --git a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBinIndexQuery.java b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBinIndexQuery.java index 979872335..c92ac3d81 100644 --- a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBinIndexQuery.java +++ b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestBinIndexQuery.java @@ -20,91 +20,147 @@ import com.basho.riak.client.api.annotations.RiakBucketName; import com.basho.riak.client.api.annotations.RiakIndex; import com.basho.riak.client.api.annotations.RiakKey; -import com.basho.riak.client.api.annotations.RiakVClock; -import com.basho.riak.client.api.cap.VClock; -import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.core.operations.itest.ITestAutoCleanupBase; import com.basho.riak.client.api.commands.indexes.BinIndexQuery; +import com.basho.riak.client.api.commands.indexes.SecondaryIndexQuery; import com.basho.riak.client.api.commands.kv.StoreValue; +import com.basho.riak.client.core.RiakFuture; import com.basho.riak.client.core.query.Location; -import com.basho.riak.client.core.query.Namespace; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.List; import java.util.concurrent.ExecutionException; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.junit.Assume; -import org.junit.Test; /** - * * @author Brian Roach + * @author Alex Moore * @since 2.0 */ -public class ITestBinIndexQuery extends ITestAutoCleanupBase +public class ITestBinIndexQuery extends ITestIndexBase { + private static final String OBJECT_KEY_BASE = "index_test_object_key"; + private static final String INDEX_NAME = "test_index"; + private static final String INDEX_KEY_BASE = "index_test_index_key"; + private static RiakClient client = new RiakClient(cluster); + + @BeforeClass + public static void setupData() throws InterruptedException + { + createIndexedPojo(0, 1); + createIndexedPojo(1, 1); + createIndexedPojo(2, 2); + } + @Test public void testMatchQuery() throws ExecutionException, InterruptedException { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); + BinIndexQuery indexQuery = new BinIndexQuery.Builder(namespace, INDEX_NAME, indexKey(1)).withKeyAndIndex(true) + .build(); + BinIndexQuery.Response indexResponse = client.execute(indexQuery); - IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key"; - ip.bucketName = bucketName.toString(); - ip.indexKey = "index_test_index_key"; - ip.value = "My Object Value!"; + assertTrue(indexResponse.hasEntries()); + assertEquals(2, indexResponse.getEntries().size()); - StoreValue sv = new StoreValue.Builder(ip).build(); - RiakFuture svFuture = client.executeAsync(sv); + boolean found = assertFirstObjectFound(indexResponse.getEntries()); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + assertTrue(found); + } - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = "index_test_index_key"; - ip2.value = "My Object Value!"; + @Test + public void testRangeQuery() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); + BinIndexQuery indexQuery = new BinIndexQuery.Builder(namespace, + INDEX_NAME, + indexKey(0), + indexKey(9)).withKeyAndIndex(true).build(); + BinIndexQuery.Response indexResponse = client.execute(indexQuery); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + assertTrue(indexResponse.hasEntries()); + assertEquals(3, indexResponse.getEntries().size()); + + boolean found = assertFirstObjectFound(indexResponse.getEntries()); + + assertTrue(found); + } + + @Test + public void testStreamingRangeQuery() throws ExecutionException, InterruptedException + { + Assume.assumeTrue(test2i); + + BinIndexQuery indexQuery = new BinIndexQuery.Builder(namespace, + INDEX_NAME, + indexKey(0), + indexKey(9)).withKeyAndIndex(true).build(); - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); + final RiakFuture streamingFuture = + client.executeAsyncStreaming(indexQuery, 200); - BinIndexQuery biq = - new BinIndexQuery.Builder(ns, "test_index", "index_test_index_key").withKeyAndIndex(true).build(); - BinIndexQuery.Response iResp = client.execute(biq); + final BinIndexQuery.Response streamingResponse = streamingFuture.get(); - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); + assertTrue(streamingResponse.hasEntries()); + boolean caught = false; + try + { + streamingResponse.getEntries(); + } + catch (IllegalStateException unused) + { + caught = true; + } + assertTrue(caught); + + final String expectedObjectKey = objectKey(1); + final String expectedIndexKey = indexKey(1); boolean found = false; - for (BinIndexQuery.Response.Entry e : iResp.getEntries()) + int size = 0; + + for (BinIndexQuery.Response.Entry e : streamingResponse) { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key")) + size++; + if (e.getRiakObjectLocation().getKey().toString().equals(expectedObjectKey)) { found = true; - assertEquals(ip.indexKey, e.getIndexKey()); + assertEquals(expectedIndexKey, e.getIndexKey()); } } assertTrue(found); + assertEquals(3, size); } - @Test - public void testRangeQuery() throws InterruptedException, ExecutionException + private boolean assertFirstObjectFound(Iterable> entries) { - Assume.assumeTrue(test2i); + final String expectedObjectKey = objectKey(1); + final String expectedIndexKey = indexKey(1); + boolean found = false; - RiakClient client = new RiakClient(cluster); + for (BinIndexQuery.Response.Entry e : entries) + { + if (e.getRiakObjectLocation().getKey().toString().equals(expectedObjectKey)) + { + found = true; + assertEquals(expectedIndexKey, e.getIndexKey()); + } + } + return found; + } + private static void createIndexedPojo(int keySuffix, int indexSuffix) throws InterruptedException + { IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key1"; - ip.bucketName = bucketName.toString(); - ip.indexKey = "index_test_index_key1"; + ip.key = objectKey(keySuffix); + ip.bucketName = namespace.getBucketNameAsString(); + ip.indexKey = indexKey(indexSuffix); ip.value = "My Object Value!"; StoreValue sv = new StoreValue.Builder(ip).build(); @@ -112,41 +168,19 @@ public void testRangeQuery() throws InterruptedException, ExecutionException svFuture.await(); assertTrue(svFuture.isSuccess()); + } - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = "index_test_index_key2"; - ip2.value = "My Object Value!"; - - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); - - svFuture.await(); - assertTrue(svFuture.isSuccess()); - - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); - BinIndexQuery biq = - new BinIndexQuery.Builder(ns, "test_index", "index_test_index_key0", "index_test_index_key9").withKeyAndIndex(true).build(); - BinIndexQuery.Response iResp = client.execute(biq); - - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); - - boolean found = false; - for (BinIndexQuery.Response.Entry e : iResp.getEntries()) - { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key1")) - { - found = true; - assertEquals(ip.indexKey, e.getIndexKey()); - } - } + private static String objectKey(int suffix) + { + return OBJECT_KEY_BASE + suffix; + } - assertTrue(found); + private static String indexKey(int suffix) + { + return INDEX_KEY_BASE + suffix; } - public static class IndexedPojo + private static class IndexedPojo { @RiakKey public String key; @@ -154,12 +188,9 @@ public static class IndexedPojo @RiakBucketName public String bucketName; - @RiakIndex(name="test_index") + @RiakIndex(name = INDEX_NAME) String indexKey; - @RiakVClock - VClock vclock; - public String value; } } diff --git a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestFullBucketRead.java b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestFullBucketRead.java index 5c5e73ae9..5da5123d4 100644 --- a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestFullBucketRead.java +++ b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestFullBucketRead.java @@ -110,7 +110,7 @@ public void readPlainTextValues() throws ExecutionException, InterruptedExceptio { assertFalse(e.hasFetchedValue()); assertNull(e.getFetchedValue()); - returnedKeys.add(e.getLocation().getKeyAsString()); + returnedKeys.add(e.getRiakObjectLocation().getKeyAsString()); } assertEquals(NUMBER_OF_TEST_VALUES, returnedKeys.size()); @@ -134,7 +134,7 @@ public void readPlainTextValuesWithReturnBody() assertTrue(e.hasFetchedValue()); final RiakObject ro = e.getFetchedValue().getValue(RiakObject.class); - final int expectedValue = Integer.parseInt(e.getLocation().getKeyAsString().substring(1)); + final int expectedValue = Integer.parseInt(e.getRiakObjectLocation().getKeyAsString().substring(1)); assertEquals("v" + expectedValue, ro.getValue().toString()); assertEquals("plain/text", ro.getContentType()); @@ -145,38 +145,41 @@ public void readPlainTextValuesWithReturnBody() public void readPlainTextValuesWithCoverageContext() throws ExecutionException, InterruptedException, UnknownHostException { - final Map results = performFBReadWithCoverageContext(false, false); + final Map results = performFBReadWithCoverageContext(false, false, false); assertEquals(NUMBER_OF_TEST_VALUES, results.size()); - for (int i=0; i results = performFBReadWithCoverageContext(false, false, true); + assertEquals(NUMBER_OF_TEST_VALUES, results.size()); + + verifyResults(results, false); } @Test public void readPlainTextValuesWithCoverageContextContinuouslyWithReturnBody() throws ExecutionException, InterruptedException, UnknownHostException { - final Map results = performFBReadWithCoverageContext(true, true); + final Map results = performFBReadWithCoverageContext(true, true, false); assertEquals(NUMBER_OF_TEST_VALUES, results.size()); - for (int i=0; i results = performFBReadWithCoverageContext(true, true, true); + assertEquals(NUMBER_OF_TEST_VALUES, results.size()); + + verifyResults(results, true); } @Test @@ -216,7 +219,31 @@ public void queryDataByUsingAnAlternateCoveragePlan() } } - private Map performFBReadWithCoverageContext(boolean withContinuation,boolean withReturnBody) + private static void verifyResults(final Map results, boolean withReturnBody ) + { + for (int i=0; i performFBReadWithCoverageContext(boolean withContinuation, boolean withReturnBody, + boolean useStreaming) throws UnknownHostException, ExecutionException, InterruptedException { final Map> chunkedKeys = new HashMap<>(); @@ -240,7 +267,7 @@ private Map performFBReadWithCoverageContext(boolean withCon for (CoverageEntry ce : coveragePlan.hostEntries(host)) { final Map> keys = - retrieveChunkedKeysForCoverageEntry(rc, ce, withContinuation, withReturnBody); + retrieveChunkedKeysForCoverageEntry(rc, ce, withContinuation, withReturnBody, useStreaming); chunkedKeys.putAll(keys); } } @@ -272,7 +299,7 @@ private Map readDataForCoverageEntry(CoverageEntry ce) Map> keys = Collections.emptyMap(); try { - keys = retrieveChunkedKeysForCoverageEntry(rc, ce, true, true); + keys = retrieveChunkedKeysForCoverageEntry(rc, ce, true, true, false); } finally { @@ -286,9 +313,11 @@ private Map readDataForCoverageEntry(CoverageEntry ce) private Map> retrieveChunkedKeysForCoverageEntry(RiakClient rc, CoverageEntry ce, boolean withContinuation, - boolean withReturnBody) + boolean withReturnBody, + boolean useStreaming) throws ExecutionException, InterruptedException { + final int pollTimeout = 100; // <-- in ms in case when useStreaming == true final Map> chunkedKeys = new HashMap<>(); if (!withContinuation) { @@ -296,8 +325,21 @@ private Map> retrieveChunkedKeysForCoverageEntry(Riak .withReturnBody(withReturnBody) .build(); - final FullBucketRead.Response readResponse = rc.execute(cmd2); - chunkedKeys.put(ce, readResponse.getEntries()); + final FullBucketRead.Response readResponse; + if (!useStreaming) + { + readResponse = rc.execute(cmd2); + } + else + { + readResponse = rc.executeAsyncStreaming(cmd2, pollTimeout).get(); + } + + assertEquals(useStreaming, readResponse.isStreaming()); + final List list = new LinkedList<>(); + readResponse.forEach( e -> list.add(e)); + + chunkedKeys.put(ce, list); } else { @@ -313,17 +355,25 @@ private Map> retrieveChunkedKeysForCoverageEntry(Riak .withContinuation(continuation) .build(); - final FullBucketRead.Response r = rc.execute(cmd2); - final List entries; + final FullBucketRead.Response r; - if (r.hasEntries()) + if (!useStreaming) { - entries = r.getEntries(); - data.addAll(entries); + r = rc.execute(cmd2); } else { - entries = Collections.EMPTY_LIST; + r = rc.executeAsyncStreaming(cmd2, pollTimeout).get(); + } + + assertEquals(useStreaming, r.isStreaming()); + + final List entries = new LinkedList<>(); + + if (r.hasEntries()) + { + r.forEach(e-> entries.add(e)); + data.addAll(entries); } logger.debug("FullBucketRead query(ce={}, token={}) returns:\n token: {}\n entries: {}", @@ -354,7 +404,7 @@ private Map transformChunkedKeysToRiakObjects(Map + * @author Alex Moore * @since 2.0 */ -public class ITestIntIndexQuery extends ITestAutoCleanupBase +public class ITestIntIndexQuery extends ITestIndexBase { + private static final String OBJECT_KEY_BASE = "index_test_object_key"; + private static final String INDEX_NAME = "test_index"; + private static final Long DUP_INDEX_KEY = 123456L; + private static RiakClient client = new RiakClient(cluster); + + @BeforeClass + public static void setupData() throws InterruptedException, ExecutionException + { + createIndexedPojo(0, DUP_INDEX_KEY); + createIndexedPojo(1, DUP_INDEX_KEY); + createIndexedPojo(2, 25L); + } + + @Test + public void testFetchThings() throws ExecutionException, InterruptedException + { + FetchValue fv = new FetchValue.Builder(new Location(namespace, objectKey(0))).build(); + final FetchValue.Response execute = client.execute(fv); + + execute.getValues(); + } + @Test public void testMatchQuery() throws InterruptedException, ExecutionException { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); + IntIndexQuery indexQuery = new IntIndexQuery.Builder(namespace, + INDEX_NAME, DUP_INDEX_KEY).withKeyAndIndex(true).build(); - IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key"; - ip.bucketName = bucketName.toString(); - ip.indexKey = 123456L; - ip.value = "My Object Value!"; + IntIndexQuery.Response indexResponse = client.execute(indexQuery); - StoreValue sv = new StoreValue.Builder(ip).build(); - RiakFuture svFuture = client.executeAsync(sv); + assertTrue(indexResponse.hasEntries()); + assertEquals(2, indexResponse.getEntries().size()); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + assertFirstObjectFound(indexResponse); + } - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = 123456L; - ip2.value = "My Object Value!"; + @Test + public void testRangeQuery() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); + IntIndexQuery indexQuery = new IntIndexQuery.Builder(namespace, + INDEX_NAME, + Long.MIN_VALUE, + Long.MAX_VALUE).withKeyAndIndex(true).build(); - svFuture.await(); - assertTrue(svFuture.isSuccess()); + IntIndexQuery.Response indexResponse = client.execute(indexQuery); + assertTrue(indexResponse.hasEntries()); + assertEquals(3, indexResponse.getEntries().size()); + + assertFirstObjectFound(indexResponse); + } + + @Test + public void testStreamingRangeQuery() throws ExecutionException, InterruptedException + { + Assume.assumeTrue(test2i); + + IntIndexQuery indexQuery = new IntIndexQuery.Builder(namespace, + INDEX_NAME, + Long.MIN_VALUE, + Long.MAX_VALUE).withKeyAndIndex(true).build(); - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); + final RiakFuture streamingFuture = + client.executeAsyncStreaming(indexQuery, 200); - IntIndexQuery iiq = - new IntIndexQuery.Builder(ns, "test_index", 123456L).withKeyAndIndex(true).build(); - IntIndexQuery.Response iResp = client.execute(iiq); + final IntIndexQuery.Response streamingResponse = streamingFuture.get(); - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); + assertTrue(streamingResponse.hasEntries()); + boolean caught = false; + try + { + streamingResponse.getEntries(); + } + catch (IllegalStateException unused) + { + caught = true; + } + assertTrue(caught); + + final String expectedObjectKey = objectKey(1); boolean found = false; - for (IntIndexQuery.Response.Entry e : iResp.getEntries()) + int size = 0; + + for (IntIndexQuery.Response.Entry e : streamingResponse) { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key")) + size++; + if (e.getRiakObjectLocation().getKey().toString().equals(expectedObjectKey)) { found = true; - assertEquals(ip.indexKey, e.getIndexKey()); + assertEquals(DUP_INDEX_KEY, e.getIndexKey()); } } assertTrue(found); + assertEquals(3, size); } - @Test - public void testRangeQuery() throws InterruptedException, ExecutionException + private void assertFirstObjectFound(IntIndexQuery.Response indexResponse) { - Assume.assumeTrue(test2i); + boolean found = false; + for (IntIndexQuery.Response.Entry e : indexResponse.getEntries()) + { + if (e.getRiakObjectLocation().getKey().toString().equals(objectKey(1))) + { + found = true; + assertEquals(DUP_INDEX_KEY, e.getIndexKey()); + } + } - RiakClient client = new RiakClient(cluster); + assertTrue(found); + } + private static void createIndexedPojo(int keySuffix, long indexKey) throws InterruptedException + { IndexedPojo ip = new IndexedPojo(); - ip.key = "index_test_object_key1"; - ip.bucketName = bucketName.toString(); - ip.indexKey = 1L; + ip.key = objectKey(keySuffix); + ip.bucketName = namespace.getBucketNameAsString(); + ip.indexKey = indexKey; ip.value = "My Object Value!"; StoreValue sv = new StoreValue.Builder(ip).build(); @@ -112,44 +171,14 @@ public void testRangeQuery() throws InterruptedException, ExecutionException svFuture.await(); assertTrue(svFuture.isSuccess()); + } - IndexedPojo ip2 = new IndexedPojo(); - ip2.key = "index_test_object_key2"; - ip2.bucketName = bucketName.toString(); - ip2.indexKey = 25L; - ip2.value = "My Object Value!"; - - sv = new StoreValue.Builder(ip2).build(); - svFuture = client.executeAsync(sv); - - svFuture.await(); - assertTrue(svFuture.isSuccess()); - - Namespace ns = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString()); - - IntIndexQuery iiq = - new IntIndexQuery.Builder(ns, "test_index", Long.MIN_VALUE, Long.MAX_VALUE) - .withKeyAndIndex(true) - .build(); - - IntIndexQuery.Response iResp = client.execute(iiq); - assertTrue(iResp.hasEntries()); - assertEquals(2, iResp.getEntries().size()); - - boolean found = false; - for (IntIndexQuery.Response.Entry e : iResp.getEntries()) - { - if (e.getRiakObjectLocation().getKey().toString().equals("index_test_object_key1")) - { - found = true; - assertEquals(ip.indexKey, e.getIndexKey()); - } - } - - assertTrue(found); + private static String objectKey(int suffix) + { + return OBJECT_KEY_BASE + suffix; } - public static class IndexedPojo + private static class IndexedPojo { @RiakKey public String key; @@ -157,12 +186,9 @@ public static class IndexedPojo @RiakBucketName public String bucketName; - @RiakIndex(name="test_index") + @RiakIndex(name = "test_index") Long indexKey; - @RiakVClock - VClock vclock; - public String value; } } diff --git a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestRawIndexQuery.java b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestRawIndexQuery.java index d0178aa2c..3fa722604 100644 --- a/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestRawIndexQuery.java +++ b/src/test/java/com/basho/riak/client/api/commands/indexes/itest/ITestRawIndexQuery.java @@ -20,33 +20,40 @@ import com.basho.riak.client.api.annotations.RiakBucketName; import com.basho.riak.client.api.annotations.RiakIndex; import com.basho.riak.client.api.annotations.RiakKey; -import com.basho.riak.client.api.annotations.RiakVClock; -import com.basho.riak.client.api.cap.VClock; -import com.basho.riak.client.api.commands.indexes.*; +import com.basho.riak.client.api.commands.indexes.BinIndexQuery; +import com.basho.riak.client.api.commands.indexes.BucketIndexQuery; +import com.basho.riak.client.api.commands.indexes.KeyIndexQuery; +import com.basho.riak.client.api.commands.indexes.RawIndexQuery; import com.basho.riak.client.api.commands.indexes.SecondaryIndexQuery.Type; import com.basho.riak.client.api.commands.kv.StoreValue; import com.basho.riak.client.core.RiakFuture; import com.basho.riak.client.core.operations.StoreOperation; -import com.basho.riak.client.core.operations.itest.ITestAutoCleanupBase; +import com.basho.riak.client.core.operations.itest.ITestBase; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.RiakObject; import com.basho.riak.client.core.query.indexes.IndexNames; import com.basho.riak.client.core.util.BinaryValue; -import org.junit.*; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; import java.util.concurrent.ExecutionException; +import java.util.stream.StreamSupport; import static org.junit.Assert.*; /** * * @author Brian Roach + * @author Alex Moore */ -public class ITestRawIndexQuery extends ITestAutoCleanupBase +public class ITestRawIndexQuery extends ITestBase { private static String sharedBucket = "ITestRawIndexQuery"; private static Namespace sharedNamespace = new Namespace(Namespace.DEFAULT_BUCKET_TYPE, sharedBucket); + private static RiakClient client = new RiakClient(cluster); @BeforeClass public static void Setup() throws ExecutionException, InterruptedException @@ -79,8 +86,6 @@ public void simpleTest() throws InterruptedException, ExecutionException setBucketNameToTestName(); - RiakClient client = new RiakClient(cluster); - BinaryValue indexKey = BinaryValue.create("index_test_index_key"); IndexedPojo ip = new IndexedPojo(); @@ -102,7 +107,7 @@ public void simpleTest() throws InterruptedException, ExecutionException RawIndexQuery.Response iResp = client.execute(biq); assertTrue(iResp.hasEntries()); - RawIndexQuery.Response.Entry first = iResp.getEntries().iterator().next(); + RawIndexQuery.Response.Entry first = iResp.getEntries().iterator().next(); assertEquals(ip.key, first.getRiakObjectLocation().getKey().toString()); assertArrayEquals(ip.indexKey, first.getIndexKey().getValue()); } @@ -112,8 +117,6 @@ public void testKeyIndexHack() throws InterruptedException, ExecutionException { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); - RawIndexQuery biq = new RawIndexQuery.Builder(sharedNamespace, IndexNames.KEY, @@ -132,8 +135,6 @@ public void testKeyIndexQuery() throws InterruptedException, ExecutionException { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); - KeyIndexQuery kq = new KeyIndexQuery.Builder(sharedNamespace, "my_key10", "my_key19").build(); final RawIndexQuery.Response kqResp = client.execute(kq); @@ -146,8 +147,6 @@ public void testBucketIndexHack() throws InterruptedException, ExecutionExceptio { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); - RawIndexQuery biq = new RawIndexQuery.Builder(sharedNamespace, IndexNames.BUCKET, Type._BUCKET, BinaryValue.create(sharedBucket)) @@ -163,8 +162,6 @@ public void testBucketIndexQuery() throws InterruptedException, ExecutionExcepti { Assume.assumeTrue(test2i); - RiakClient client = new RiakClient(cluster); - BucketIndexQuery bq = new BucketIndexQuery.Builder(sharedNamespace).build(); final BinIndexQuery.Response bqResp = client.execute(bq); @@ -172,7 +169,62 @@ public void testBucketIndexQuery() throws InterruptedException, ExecutionExcepti assertEquals(100, bqResp.getEntries().size()); } - public static class IndexedPojo + @Test + public void testBucketIndexQueryStreaming() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); + + BucketIndexQuery bq = new BucketIndexQuery.Builder(sharedNamespace).build(); + + final RiakFuture indexResult = + client.executeAsyncStreaming(bq, 100); + + final BinIndexQuery.Response streamingResponse = indexResult.get(); + + assertTrue(streamingResponse.hasEntries()); + + boolean caught = false; + try + { + streamingResponse.getEntries(); + } + catch (IllegalStateException unused) + { + caught = true; + } + assertTrue(caught); + + assertEquals(100, StreamSupport.stream(streamingResponse.spliterator(), false).count()); + + // Assert everything was consumed + assertFalse(streamingResponse.hasEntries()); + } + + @Test + public void testIndexQueryStreamingContinuations() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); + + BucketIndexQuery bq = new BucketIndexQuery.Builder(sharedNamespace).withMaxResults(50).withPaginationSort(true).build(); + + final RiakFuture indexResult = + client.executeAsyncStreaming(bq, 100); + + final BinIndexQuery.Response streamingResponse = indexResult.get(); + + assertTrue(streamingResponse.hasEntries()); + assertEquals(50, StreamSupport.stream(streamingResponse.spliterator(), false).count()); + assertTrue(streamingResponse.hasContinuation()); + assertNotNull(streamingResponse.getContinuation()); + + final BinIndexQuery queryInfo = indexResult.getQueryInfo(); + assertEquals(bq, queryInfo); + + // Assert everything was consumed + assertFalse(streamingResponse.hasEntries()); + } + + private static class IndexedPojo { @RiakKey public String key; @@ -183,9 +235,6 @@ public static class IndexedPojo @RiakIndex(name="test_index_bin") byte[] indexKey; - @RiakVClock - VClock vclock; - public String value; } } diff --git a/src/test/java/com/basho/riak/client/api/commands/itest/ITestBucketMapReduce.java b/src/test/java/com/basho/riak/client/api/commands/itest/ITestBucketMapReduce.java index f88e26dfb..dbd056547 100644 --- a/src/test/java/com/basho/riak/client/api/commands/itest/ITestBucketMapReduce.java +++ b/src/test/java/com/basho/riak/client/api/commands/itest/ITestBucketMapReduce.java @@ -18,7 +18,6 @@ import com.basho.riak.client.api.RiakClient; import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.core.operations.itest.ITestAutoCleanupBase; import com.basho.riak.client.api.commands.buckets.StoreBucketProperties; import com.basho.riak.client.api.commands.kv.StoreValue; import com.basho.riak.client.api.commands.mapreduce.BucketMapReduce; @@ -35,11 +34,13 @@ import com.basho.riak.client.core.query.functions.Function; import com.basho.riak.client.core.util.BinaryValue; import com.fasterxml.jackson.databind.node.ArrayNode; + import java.util.concurrent.ExecutionException; import org.junit.*; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; /** @@ -117,6 +118,19 @@ public void erlangBucketMRTestType() throws InterruptedException, ExecutionExcep erlangBucketMR(mapReduceBucketType.toString()); } + @Test + public void erlangBucketMRDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + streamingErlangBucketMR(Namespace.DEFAULT_BUCKET_TYPE); + } + + @Test + public void erlangBucketMRTestTypeStreaming() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(testBucketType); + streamingErlangBucketMR(mapReduceBucketType.toString()); + } + private void erlangBucketMR(String bucketType) throws InterruptedException, ExecutionException { Namespace ns = new Namespace(bucketType, mrBucketName); @@ -140,6 +154,67 @@ private void erlangBucketMR(String bucketType) throws InterruptedException, Exec assertEquals(199, result.get(199).asInt()); } + private void streamingErlangBucketMR(String bucketType) throws InterruptedException, ExecutionException + { + Namespace ns = new Namespace(bucketType, mrBucketName); + BucketMapReduce bmr = + new BucketMapReduce.Builder() + .withNamespace(ns) + .withMapPhase(Function.newErlangFunction("riak_kv_mapreduce", "map_object_value"), false) + .withReducePhase(Function.newErlangFunction("riak_kv_mapreduce", "reduce_string_to_integer"), false) + .withReducePhase(Function.newErlangFunction("riak_kv_mapreduce", "reduce_sort"), true) + .build(); + + final RiakFuture streamingFuture = + client.executeAsyncStreaming(bmr, 10); + + boolean found42 = false; + boolean found199 = false; + int count = 0; + + final MapReduce.Response streamingResponse = streamingFuture.get(); + assertTrue(streamingResponse.isStreaming()); + // The streaming query should return many results which are JSON arrays, each + // containing a piece of the array [0-199]. + // Streaming result would look like: [[0], [1,2,3], ... [..., 199]], with the outer + // array being the different response chunks streaming in. + for (MapReduce.Response response : streamingResponse) + { + int phaseSize = response.getResultsFromAllPhases().size(); + + if (phaseSize == 0) + { + continue; + } + + count += phaseSize; + + final ArrayNode result = response.getResultForPhase(2); + if (result == null) + { + continue; + } + + final String valuesString = result.toString(); + + if (!found42) + { + found42 = valuesString.contains("42"); + } + if (!found199) + { + found199 = valuesString.contains("199"); + } + } + + assertEquals(200, count); + assertTrue(found42); + assertTrue(found199); + + // Assert that we have consumed the responses, and none are left. + assertFalse(streamingFuture.get().iterator().hasNext()); + } + @Test public void JsBucketMRDefaultType() throws InterruptedException, ExecutionException { diff --git a/src/test/java/com/basho/riak/client/core/ChunkedResponseIteratorTest.java b/src/test/java/com/basho/riak/client/core/ChunkedResponseIteratorTest.java new file mode 100644 index 000000000..e79d3b584 --- /dev/null +++ b/src/test/java/com/basho/riak/client/core/ChunkedResponseIteratorTest.java @@ -0,0 +1,237 @@ +package com.basho.riak.client.core; + +import com.basho.riak.client.api.commands.ChunkedResponseIterator; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.mockito.Mock; +import org.mockito.stubbing.Answer; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TransferQueue; + +import static org.junit.Assert.*; +import static org.powermock.api.mockito.PowerMockito.when; +import static org.mockito.Matchers.any; + +/** + * @author Alex Moore + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(PBStreamingFutureOperation.class) +public class ChunkedResponseIteratorTest +{ + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Mock + private TransferQueue fakeQueue; + + @Mock + private PBStreamingFutureOperation coreFuture; + + private final int timeout = 1000; + + @Before + public void initializeMOcks() + { + when(coreFuture.getResultsQueue()).thenReturn(fakeQueue); + } + + /* + NB: Disable this test if you want to use code coverage tools. + The Thread interrupt it generates triggers a shutdown hook race bug in Java, + which then doesn't allow EMMA to cleanly shutdown. + https://bugs.openjdk.java.net/browse/JDK-8154017 + */ + @Test + public void testInterruptedExceptionUponInitialisation() throws InterruptedException + { + Thread testThread = new Thread(() -> + { + try + { + when(fakeQueue.poll(any(Long.class), any(TimeUnit.class))) + .thenThrow(new InterruptedException()) + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Arrays.asList(1,2,3,4,5).iterator(); + } + }); + + // ChunkedResponseIterator polls the response queue when created, + // so we'll use that to simulate a Thread interrupt. + new ChunkedResponseIterator<>(coreFuture, + timeout, + Long::new, + FakeResponse::iterator); + } + catch (InterruptedException e) + { + // Mocking TransferQueue::poll(timeout) requires this CheckedException be dealt with + // If we actually catch one here we've failed at our jobs. + fail(e.getMessage()); + } + + assertTrue(Thread.currentThread().isInterrupted()); + }); + + testThread.start(); + testThread.join(); + assertFalse(Thread.currentThread().isInterrupted()); + } + + @Test + public void testInterruptedExceptionUponNextChunkLoad() throws InterruptedException + { + Thread testThread = new Thread(() -> + { + try + { + when(fakeQueue.poll(any(Long.class), any(TimeUnit.class))) + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Collections.singletonList(1).iterator(); + } + }) + .thenThrow(new InterruptedException()) + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Collections.singletonList(2).iterator(); + } + }); + + ChunkedResponseIterator chunkedResponseIterator = + new ChunkedResponseIterator<>(coreFuture, timeout, Long::new, FakeResponse::iterator); + + assertTrue(chunkedResponseIterator.hasNext()); + assertEquals(new Long(1), chunkedResponseIterator.next()); + // Should hit InterruptedException here, and then take care of it. + assertTrue(chunkedResponseIterator.hasNext()); + assertEquals(new Long(2), chunkedResponseIterator.next()); + } + catch (InterruptedException e) + { + // Mocking TransferQueue::poll(timeout) requires this CheckedException be dealt with + // If we actually catch one here we've failed at our jobs. + fail(e.getMessage()); + } + + assertTrue(Thread.currentThread().isInterrupted()); + }); + + testThread.start(); + testThread.join(); + assertFalse(Thread.currentThread().isInterrupted()); + } + + @Test + public void testConcurrentDoneAndInterruptedException() throws InterruptedException + { + Thread testThread = new Thread(() -> + { + try + { + when(coreFuture.isDone()).thenReturn(false); + + when(fakeQueue.poll(any(Long.class), any(TimeUnit.class))) + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Collections.singletonList(1).iterator(); + } + }) + .thenAnswer(invocationOnMock -> + { + when(coreFuture.isDone()).thenReturn(true); + when(fakeQueue.isEmpty()).thenReturn(true); + throw new InterruptedException(); + }); + + ChunkedResponseIterator chunkedResponseIterator = + new ChunkedResponseIterator<>(coreFuture, timeout, Long::new, FakeResponse::iterator); + + assertTrue(chunkedResponseIterator.hasNext()); + assertEquals(new Long(1), chunkedResponseIterator.next()); + // InterruptedException should happen when we try to load the next chunk, + // But the catch + next attempt to load a chunk should check to see if we're done. + assertFalse(chunkedResponseIterator.hasNext()); + + } + catch (InterruptedException e) + { + // Mocking TransferQueue::poll(timeout) requires this CheckedException be dealt with + // If we actually catch one here we've failed at our jobs. + fail(e.getMessage()); + } + + assertTrue(Thread.currentThread().isInterrupted()); + }); + + testThread.start(); + testThread.join(); + assertFalse(Thread.currentThread().isInterrupted()); + } + + @Test(timeout = 5000) + public void checkProperIterationThroughChunkedResponse() throws InterruptedException { + when(fakeQueue.poll(any(Long.class), any(TimeUnit.class))) + // Simulate first chunk + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Arrays.asList(1,2).iterator(); + } + }) + // Simulate next chunk + .thenReturn( + new FakeResponse() { + @Override + public Iterator iterator() + { + return Arrays.asList(3,4).iterator(); + } + }) + // Simulate completion + .thenAnswer(invocationOnMock -> + { + when(coreFuture.isDone()).thenReturn(true); + when(fakeQueue.isEmpty()).thenReturn(true); + return null; + }); + + final ChunkedResponseIterator iterator = + new ChunkedResponseIterator<>(coreFuture, 50, Long::new, FakeResponse::iterator); + + assertEquals(1l, iterator.next().longValue()); + assertEquals(2l, iterator.next().longValue()); + assertEquals(3l, iterator.next().longValue()); + assertEquals(4l, iterator.next().longValue()); + + assertFalse(iterator.hasNext()); + + exception.expect(NoSuchElementException.class); + iterator.next(); + } + + static abstract class FakeResponse implements Iterable {} +} diff --git a/src/test/java/com/basho/riak/client/core/operations/itest/ITestCoveragePlan.java b/src/test/java/com/basho/riak/client/core/operations/itest/ITestCoveragePlan.java index 02c26ce9e..ed5b5090a 100644 --- a/src/test/java/com/basho/riak/client/core/operations/itest/ITestCoveragePlan.java +++ b/src/test/java/com/basho/riak/client/core/operations/itest/ITestCoveragePlan.java @@ -132,7 +132,7 @@ public void fetchAllDataByUsingCoverageContext() throws ExecutionException, Inte logger.info(sbld.toString()); } - final Map> chunkedKeys + final Map>> chunkedKeys = new HashMap<>(); for (HostAndPort host: response.hosts()) @@ -170,7 +170,7 @@ public void fetchAllDataByUsingCoverageContext() throws ExecutionException, Inte } final Set keys = new HashSet<>(NUMBER_OF_TEST_VALUES); - for (Map.Entry> e: chunkedKeys.entrySet()) + for (Map.Entry>> e: chunkedKeys.entrySet()) { final CoverageEntry ce = e.getKey(); if (e.getValue().isEmpty()) diff --git a/src/test/java/com/basho/riak/client/core/operations/itest/ITestListBucketsOperation.java b/src/test/java/com/basho/riak/client/core/operations/itest/ITestListBucketsOperation.java index 36f8993ea..50acaa8ab 100644 --- a/src/test/java/com/basho/riak/client/core/operations/itest/ITestListBucketsOperation.java +++ b/src/test/java/com/basho/riak/client/core/operations/itest/ITestListBucketsOperation.java @@ -13,11 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.basho.riak.client.core.operations.itest; import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.core.RiakFutureListener; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.ListBucketsOperation; +import com.basho.riak.client.core.operations.ListKeysOperation; import com.basho.riak.client.core.operations.StoreOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; @@ -26,12 +28,12 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; -import java.util.concurrent.CountDownLatch; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.junit.Assert.assertTrue; import static org.junit.Assume.assumeTrue; @@ -42,136 +44,169 @@ */ public class ITestListBucketsOperation extends ITestAutoCleanupBase { + private final String defaultBucketType = Namespace.DEFAULT_BUCKET_TYPE; + private final String namedBucketType = ITestBase.bucketType.toStringUtf8(); + @Test public void testListBucketsDefaultType() throws InterruptedException, ExecutionException { - testListBuckets(Namespace.DEFAULT_BUCKET_TYPE); + testBucketList(defaultBucketType, 1); + } + + @Test + public void testListBucketsDefaultTypeStreaming() throws ExecutionException, InterruptedException + { + testBucketListStreaming(namedBucketType, 1); } @Test public void testListBucketsTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testListBuckets(bucketType.toString()); + testBucketList(namedBucketType, 1); } - private void testListBuckets(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testListBucketsTestTypeStreaming() throws InterruptedException, ExecutionException { - // Empty buckets do not show up - final BinaryValue key = BinaryValue.unsafeCreate("my_key".getBytes()); - final String value = "{\"value\":\"value\"}"; - - RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); - Location location = new Location(new Namespace(bucketType, bucketName.toString()), key); - StoreOperation storeOp = - new StoreOperation.Builder(location) - .withContent(rObj) - .build(); - - cluster.execute(storeOp); - storeOp.get(); - - ListBucketsOperation listOp = new ListBucketsOperation.Builder() - .withBucketType(BinaryValue.createFromUtf8(bucketType)) - .build(); - cluster.execute(listOp); - List bucketList = listOp.get().getBuckets(); - assertTrue(bucketList.size() > 0); - - boolean found = false; - for (BinaryValue baw : bucketList) - { - if (baw.toString().equals(bucketName.toString())) - { - found = true; - } - } - - assertTrue(found); + assumeTrue(testBucketType); + testBucketListStreaming(namedBucketType, 1); } @Test public void testLargeBucketListDefaultType() throws InterruptedException, ExecutionException { - testLargeBucketList(Namespace.DEFAULT_BUCKET_TYPE); + testBucketList(defaultBucketType, 10); + } + + @Test + public void testLargeBucketListDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + assumeTrue(testBucketType); + testBucketListStreaming(defaultBucketType, 10); } @Test public void testLargeBucketListTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testLargeBucketList(bucketType.toString()); + testBucketList(namedBucketType, 10); } - private void testLargeBucketList(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testLargeBucketListTestTypeStreaming() throws InterruptedException, ExecutionException { - final BinaryValue key = BinaryValue.unsafeCreate("my_key".getBytes()); - final String value = "{\"value\":\"value\"}"; - final Semaphore semaphore = new Semaphore(10); - final CountDownLatch latch = new CountDownLatch(1); - final int bucketCount = 10; - final List bucketNames = new ArrayList<>(); + assumeTrue(testBucketType); + testBucketListStreaming(namedBucketType, 10); + } - RiakFutureListener listener = - new RiakFutureListener() - { - private final AtomicInteger received = new AtomicInteger(); - - @Override - public void handle(RiakFuture f) - { - try - { - f.get(); - } - catch (InterruptedException | ExecutionException ex) - { - throw new RuntimeException(ex); - } - - semaphore.release(); - received.incrementAndGet(); - if (received.intValue() == bucketCount) - { - latch.countDown(); - } - } - }; + private void testBucketList(String bucketType, int bucketCount) throws InterruptedException, ExecutionException + { + final List expectedBuckets = storeObjects(bucketType, bucketCount); - for (int i = 0; i < bucketCount; i++) + List actualBucketNames = getAllBucketListResults(bucketType); + + assertContainsAll(expectedBuckets, actualBucketNames); + + resetAndEmptyBuckets(bucketType, expectedBuckets); + } + + private void testBucketListStreaming(String bucketType, int bucketCount) throws InterruptedException, ExecutionException + { + final List expectedBuckets = storeObjects(bucketType, bucketCount); + final ListBucketsOperation listOp = new ListBucketsOperation.Builder() + .withBucketType(BinaryValue.createFromUtf8(bucketType)) + .streamResults(true) + .build(); + + final StreamingRiakFuture execute = cluster.execute(listOp); + final BlockingQueue resultsQueue = execute.getResultsQueue(); + + List actualBuckets = new LinkedList<>(); + int timeouts = 0; + + while (!execute.isDone()) { - String testBucketName = bucketName.toString() + i; - bucketNames.add(BinaryValue.create(testBucketName)); + final ListBucketsOperation.Response response = resultsQueue.poll(5, TimeUnit.MILLISECONDS); - Namespace ns = new Namespace(bucketType, testBucketName); - RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); - Location location = new Location(ns, key); - StoreOperation storeOp = - new StoreOperation.Builder(location) - .withContent(rObj) - .build(); - storeOp.addListener(listener); - semaphore.acquire(); - cluster.execute(storeOp); + if (response != null) + { + actualBuckets.addAll(response.getBuckets()); + continue; + } + + timeouts++; + if (timeouts == 10) + { + break; + } } - latch.await(2, TimeUnit.MINUTES); + execute.await(); - ListBucketsOperation listOp = new ListBucketsOperation.Builder() - .withBucketType(BinaryValue.createFromUtf8(bucketType)) - .build(); - cluster.execute(listOp); - List bucketList = listOp.get().getBuckets(); + // Grab any last buckets that came in on the last message + actualBuckets.addAll(resultsQueue.stream() + .map(ListBucketsOperation.Response::getBuckets) + .flatMap(List::stream) + .collect(Collectors.toList())); + resultsQueue.clear(); + + assertTrue(resultsQueue.isEmpty()); - for (BinaryValue name : bucketNames) + assertContainsAll(expectedBuckets, actualBuckets); + + resetAndEmptyBuckets(bucketType, expectedBuckets); + } + + private void assertContainsAll(List expectedSet, List actualSet) + { + for (BinaryValue name : expectedSet) { - assertTrue(bucketList.contains(name)); + assertTrue(actualSet.contains(name)); } + } - for (BinaryValue name : bucketNames) + private void resetAndEmptyBuckets(String bucketType, List expectedBucketNames) + throws InterruptedException, ExecutionException + { + for (BinaryValue name : expectedBucketNames) { Namespace ns = new Namespace(bucketType, name.toString()); resetAndEmptyBucket(ns); } } + + private List getAllBucketListResults(String bucketType) throws InterruptedException, ExecutionException + { + final ListBucketsOperation listOp = new ListBucketsOperation.Builder() + .withBucketType(BinaryValue.createFromUtf8(bucketType)) + .build(); + cluster.execute(listOp); + return listOp.get().getBuckets(); + } + + private List storeObjects(String bucketType, int bucketCount) throws InterruptedException + { + final List bucketNames = new ArrayList<>(); + final BinaryValue key = BinaryValue.unsafeCreate("my_key".getBytes()); + final String value = "{\"value\":\"value\"}"; + final RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); + + for (int i = 0; i < bucketCount; i++) + { + final String testBucketName = testName.getMethodName() + "_" + i; + bucketNames.add(BinaryValue.create(testBucketName)); + + final Location location = new Location(new Namespace(bucketType, testBucketName), key); + final StoreOperation storeOp = new StoreOperation.Builder(location) + .withContent(rObj) + .build(); + + final RiakFuture execute = cluster.execute(storeOp); + execute.await(); + assertTrue(execute.isSuccess()); + } + + return bucketNames; + } } diff --git a/src/test/java/com/basho/riak/client/core/operations/itest/ITestListKeysOperation.java b/src/test/java/com/basho/riak/client/core/operations/itest/ITestListKeysOperation.java index fa8fc9699..f13f1b6d4 100644 --- a/src/test/java/com/basho/riak/client/core/operations/itest/ITestListKeysOperation.java +++ b/src/test/java/com/basho/riak/client/core/operations/itest/ITestListKeysOperation.java @@ -13,173 +13,226 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.basho.riak.client.core.operations.itest; import com.basho.riak.client.core.RiakFuture; -import com.basho.riak.client.core.RiakFutureListener; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.ListKeysOperation; import com.basho.riak.client.core.operations.StoreOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.RiakObject; import com.basho.riak.client.core.util.BinaryValue; +import org.junit.Test; + +import java.util.LinkedList; import java.util.List; -import java.util.concurrent.CountDownLatch; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.*; -import static org.junit.Assume.assumeTrue; -import org.junit.Ignore; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; /** * * @author Brian Roach + * @author Alex Moore */ public class ITestListKeysOperation extends ITestBase { - private Logger logger = LoggerFactory.getLogger("ITestListKeysOperation"); + private final String defaultBucketType = Namespace.DEFAULT_BUCKET_TYPE; + private final String namedBucketType = ITestBase.bucketType.toStringUtf8(); @Test public void testListNoKeysDefaultType() throws InterruptedException, ExecutionException { - testListNoKeys(Namespace.DEFAULT_BUCKET_TYPE); + testListNoKeys(defaultBucketType); + } + + @Test + public void testListNoKeysDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + testListKeysStreaming(defaultBucketType, 0); } @Test public void testListNoKeysTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testListNoKeys(bucketType.toString()); + testListNoKeys(namedBucketType); } - private void testListNoKeys(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testListNoKeysTestTypeStreaming() throws InterruptedException, ExecutionException { - Namespace ns = new Namespace(bucketType, bucketName.toString() + "_1"); - ListKeysOperation klistOp = new ListKeysOperation.Builder(ns).build(); - cluster.execute(klistOp); - List kList = klistOp.get().getKeys(); - assertTrue(kList.isEmpty()); - resetAndEmptyBucket(ns); + assumeTrue(testBucketType); + testListKeysStreaming(namedBucketType, 0); } @Test public void testListKeyDefaultType() throws InterruptedException, ExecutionException { - testListKey(Namespace.DEFAULT_BUCKET_TYPE); + testListSingleKey(defaultBucketType); + } + + @Test + public void testListKeyDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + testListKeysStreaming(defaultBucketType, 1); } @Test public void testListKeyTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testListKey(bucketType.toString()); + testListSingleKey(namedBucketType); } - private void testListKey(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testListKeyTestTypeStreaming() throws InterruptedException, ExecutionException { - final Namespace ns = new Namespace(bucketType, bucketName.toString() + "_2"); - final BinaryValue key = BinaryValue.unsafeCreate("my_key".getBytes()); - final String value = "{\"value\":\"value\"}"; - - RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); - Location location = new Location(ns, key); - StoreOperation storeOp = - new StoreOperation.Builder(location) - .withContent(rObj) - .build(); - - cluster.execute(storeOp); - storeOp.get(); - - ListKeysOperation klistOp = new ListKeysOperation.Builder(ns).build(); - cluster.execute(klistOp); - List kList = klistOp.get().getKeys(); - - assertEquals(kList.size(), 1); - assertEquals(kList.get(0), key); - resetAndEmptyBucket(ns); + assumeTrue(testBucketType); + testListKeysStreaming(namedBucketType, 1); } @Test public void testLargeKeyListDefaultType() throws InterruptedException, ExecutionException { - testLargeKeyList(Namespace.DEFAULT_BUCKET_TYPE); + testManyKeyList(defaultBucketType, 1000); + } + + @Test + public void testLargeKeyListDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + testListKeysStreaming(defaultBucketType, 1000); } @Test public void testLargeKeyListTestType() throws InterruptedException, ExecutionException { assumeTrue(testBucketType); - testLargeKeyList(bucketType.toString()); + testManyKeyList(namedBucketType, 1000); } - private void testLargeKeyList(String bucketType) throws InterruptedException, ExecutionException + @Test + public void testLargeKeyListTestTypeStreaming() throws InterruptedException, ExecutionException { - final String baseKey = "my_key"; - final String value = "{\"value\":\"value\"}"; - final Namespace ns = new Namespace(bucketType, bucketName.toString() + "_3"); - final Semaphore semaphore = new Semaphore(10); - final CountDownLatch latch = new CountDownLatch(1); - final int expected = 100; + assumeTrue(testBucketType); + testListKeysStreaming(namedBucketType, 1000); + } - RiakFutureListener listener = - new RiakFutureListener() + private void testListNoKeys(String bucketType) throws InterruptedException, ExecutionException + { + final Namespace ns = setupBucket(bucketType, 0); + + final List kList = getAllKeyListResults(ns); + + assertTrue(kList.isEmpty()); + resetAndEmptyBucket(ns); + } + + private void testListSingleKey(String bucketType) throws InterruptedException, ExecutionException + { + final Namespace ns = setupBucket(bucketType, 1); + + final List kList = getAllKeyListResults(ns); + + assertEquals(kList.size(), 1); + assertEquals(kList.get(0), createKeyName(0)); + resetAndEmptyBucket(ns); + } + + private void testManyKeyList(String bucketType, int numExpected) throws InterruptedException, ExecutionException + { + final Namespace ns = setupBucket(bucketType, numExpected); + + final List kList = getAllKeyListResults(ns); + + assertEquals(numExpected, kList.size()); + + resetAndEmptyBucket(ns); + } + + private void testListKeysStreaming(String bucketType, int numExpected) throws InterruptedException, ExecutionException + { + final Namespace ns = setupBucket(bucketType, numExpected); + + final ListKeysOperation slko = new ListKeysOperation.Builder(ns).streamResults(true).build(); + final StreamingRiakFuture execute = cluster.execute(slko); + + final BlockingQueue resultsQueue = execute.getResultsQueue(); + List actualKeys = new LinkedList<>(); + int timeouts = 0; + + while (!execute.isDone()) + { + final ListKeysOperation.Response response = resultsQueue.poll(5, TimeUnit.MILLISECONDS); + + if (response != null) { - private final AtomicInteger received = new AtomicInteger(); - - @Override - public void handle(RiakFuture f) - { - try - { - f.get(); - semaphore.release(); - received.incrementAndGet(); - - if (expected == received.intValue()) - { - logger.debug("Executing ListKeys"); - latch.countDown(); - } - } - catch (InterruptedException | ExecutionException ex) - { - throw new RuntimeException(ex); - } - } - }; - - logger.debug("Inserting data"); + actualKeys.addAll(response.getKeys()); + continue; + } - for (int i = 0; i < expected; i++) + timeouts++; + if (timeouts == 10) + { + break; + } + } + + // Grab any last buckets that came in on the last message + for (ListKeysOperation.Response response : resultsQueue) { - semaphore.acquire(); - BinaryValue key = BinaryValue.unsafeCreate((baseKey + i).getBytes()); - RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); - Location location = new Location(ns, key); - StoreOperation storeOp = - new StoreOperation.Builder(location) - .withContent(rObj) - .build(); - - storeOp.addListener(listener); - cluster.execute(storeOp); + actualKeys.addAll(response.getKeys()); } - latch.await(2, TimeUnit.MINUTES); + assertEquals(numExpected, actualKeys.size()); - ListKeysOperation klistOp = new ListKeysOperation.Builder(ns).build(); + ITestBase.resetAndEmptyBucket(ns); + } + + private List getAllKeyListResults(Namespace ns) throws InterruptedException, ExecutionException + { + final ListKeysOperation klistOp = new ListKeysOperation.Builder(ns).build(); cluster.execute(klistOp); - List kList; - kList = klistOp.get().getKeys(); - assertEquals(expected, kList.size()); + return klistOp.get().getKeys(); + } - resetAndEmptyBucket(ns); + private Namespace setupBucket(String bucketType, int numExpected) throws InterruptedException + { + final Namespace ns = new Namespace(bucketType, bucketName + "_" + testName.getMethodName()); + storeObjects(ns, numExpected); + return ns; + } + + private void storeObjects(Namespace ns, int expected) throws InterruptedException + { + final String value = "{\"value\":\"value\"}"; + final RiakObject rObj = new RiakObject().setValue(BinaryValue.create(value)); + + for (int i = 0; i < expected; i++) + { + final BinaryValue key = createKeyName(i); + + final Location location = new Location(ns, key); + final StoreOperation storeOp = + new StoreOperation.Builder(location) + .withContent(rObj) + .build(); + + final RiakFuture execute = cluster.execute(storeOp); + execute.await(); + assertTrue(execute.isSuccess()); + } + } + + private BinaryValue createKeyName(int i) + { + final String keyBase = "my_key"; + return BinaryValue.unsafeCreate((keyBase + i).getBytes()); } } diff --git a/src/test/java/com/basho/riak/client/core/operations/itest/ITestMapReduceOperation.java b/src/test/java/com/basho/riak/client/core/operations/itest/ITestMapReduceOperation.java index 31936c492..8336ae68b 100644 --- a/src/test/java/com/basho/riak/client/core/operations/itest/ITestMapReduceOperation.java +++ b/src/test/java/com/basho/riak/client/core/operations/itest/ITestMapReduceOperation.java @@ -15,19 +15,26 @@ */ package com.basho.riak.client.core.operations.itest; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.MapReduceOperation; import com.basho.riak.client.core.operations.StoreOperation; import com.basho.riak.client.core.query.Location; import com.basho.riak.client.core.query.Namespace; import com.basho.riak.client.core.query.RiakObject; import com.basho.riak.client.core.util.BinaryValue; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ArrayNode; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.util.LinkedHashMap; import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TransferQueue; import static org.junit.Assert.*; import static org.junit.Assume.assumeTrue; @@ -37,8 +44,31 @@ * @author Brian Roach * @since 2.0 */ -public class ITestMapReduceOperation extends ITestAutoCleanupBase +public class ITestMapReduceOperation extends ITestBase { + @BeforeClass + public static void setup() throws ExecutionException, InterruptedException + { + insertData(new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString())); + + if (testBucketType) + { + insertData(new Namespace(bucketType, bucketName)); + } + Thread.sleep(2000); + } + + @AfterClass + public static void cleanup() throws ExecutionException, InterruptedException + { + resetAndEmptyBucket(new Namespace(Namespace.DEFAULT_BUCKET_TYPE, bucketName.toString())); + + if (testBucketType) + { + resetAndEmptyBucket(new Namespace(bucketType, bucketName)); + } + } + @Test public void testBasicMRDefaultType() throws InterruptedException, ExecutionException, IOException { @@ -58,7 +88,120 @@ public void testBasicMRTestType() throws InterruptedException, ExecutionExceptio assertEquals(Integer.valueOf(8), resultMap.get("the")); } + @Test + public void testBasicStreamingMRTestType() throws IOException, InterruptedException + { + assumeTrue(testBucketType); + Map> phaseResultMap = testBasicStreamingMR(new Namespace(bucketType, bucketName)); + Map resultMap = phaseResultMap.get(1); + assertNotNull(resultMap.containsKey("the")); + assertEquals(Integer.valueOf(8), resultMap.get("the")); + + } + private Map testBasicMR(Namespace namespace) throws InterruptedException, ExecutionException, IOException + { + final ObjectMapper objectMapper = new ObjectMapper(); + String query = createMapReduceQuery(namespace); + + MapReduceOperation mrOp = + new MapReduceOperation.Builder(BinaryValue.unsafeCreate(query.getBytes())) + .build(); + + cluster.execute(mrOp); + mrOp.await(); + assertTrue(mrOp.isSuccess()); + ArrayNode resultList = mrOp.get().getResults().get(1); + + // The query should return one result which is a JSON array containing a + // single JSON object that is a asSet of word counts. + assertEquals(resultList.size(), 1); + + String json = resultList.get(0).toString(); + @SuppressWarnings("unchecked") + Map resultMap = objectMapper.readValue(json, Map.class); + + return resultMap; + } + + private Map> testBasicStreamingMR(Namespace namespace) throws InterruptedException, IOException + { + final String query = createMapReduceQuery(namespace); + MapReduceOperation mrOp = new MapReduceOperation.Builder(BinaryValue.unsafeCreate(query.getBytes())) + .streamResults(true) + .build(); + + final StreamingRiakFuture streamFuture = cluster.execute(mrOp); + + final TransferQueue resultsQueue = streamFuture.getResultsQueue(); + + Map> resultMap = new LinkedHashMap<>(); + + while (!streamFuture.isDone()) + { + final MapReduceOperation.Response response = resultsQueue.poll(10, TimeUnit.MILLISECONDS); + if (response == null) + { + continue; + } + mergeWordCountMaps(resultMap, response); + } + + for (MapReduceOperation.Response response : resultsQueue) + { + mergeWordCountMaps(resultMap, response); + } + + return resultMap; + } + + private void mergeWordCountMaps(Map> resultMap, + MapReduceOperation.Response result) throws IOException + { + final ObjectMapper objectMapper = new ObjectMapper(); + + for (Integer phaseNumber : result.getResults().keySet()) + { + resultMap.putIfAbsent(phaseNumber, new LinkedHashMap<>()); + final Map currentPhaseMap = resultMap.get(phaseNumber); + + final ArrayNode resultList = result.getResults().get(phaseNumber); + final String wordCountMapJsonString = resultList.get(0).toString(); + @SuppressWarnings("unchecked") + Map wordCountMap = objectMapper.readValue(wordCountMapJsonString, Map.class); + + for (String wordCountKey: wordCountMap.keySet()) + { + if (currentPhaseMap.containsKey(wordCountKey)) + { + final int newWordCountSum = currentPhaseMap.get(wordCountKey) + wordCountMap.get(wordCountKey); + currentPhaseMap.put(wordCountKey, newWordCountSum); + } + else + { + currentPhaseMap.put(wordCountKey, wordCountMap.get(wordCountKey)); + } + } + } + } + + private String createMapReduceQuery(Namespace namespace) + { + final String bucketType = namespace.getBucketTypeAsString(); + final String bucketName = namespace.getBucketNameAsString(); + + return "{\"inputs\":[[\"" + bucketName + "\",\"p1\",\"\",\"" + bucketType + "\"]," + + "[\"" + bucketName + "\",\"p2\",\"\",\"" + bucketType + "\"]," + + "[\"" + bucketName + "\",\"p3\",\"\",\"" + bucketType + "\"]]," + + "\"query\":[{\"map\":{\"language\":\"javascript\",\"source\":\"" + + "function(v) {var m = v.values[0].data.toLowerCase().match(/\\w*/g); var r = [];" + + "for (var i in m) {if (m[i] != '') {var o = {};o[m[i]] = 1;r.push(o);}}return r;}" + + "\"}},{\"reduce\":{\"language\":\"javascript\",\"source\":\"" + + "function(v) {var r = {};for (var i in v) {for(var w in v[i]) {if (w in r) r[w] += v[i][w];" + + "else r[w] = v[i][w];}}return [r];}\"}}]}"; + } + + private static void insertData(Namespace namespace) throws InterruptedException, ExecutionException { RiakObject obj = new RiakObject(); @@ -103,40 +246,5 @@ private Map testBasicMR(Namespace namespace) throws Interrupted cluster.execute(storeOp); storeOp.get(); - - String bName = namespace.getBucketNameAsString(); - String bType = namespace.getBucketTypeAsString(); - - String query = "{\"inputs\":[[\"" + bName + "\",\"p1\",\"\",\"" + bType + "\"]," + - "[\"" + bName + "\",\"p2\",\"\",\"" + bType + "\"]," + - "[\"" + bName + "\",\"p3\",\"\",\"" + bType + "\"]]," + - "\"query\":[{\"map\":{\"language\":\"javascript\",\"source\":\"" + - "function(v) {var m = v.values[0].data.toLowerCase().match(/\\w*/g); var r = [];" + - "for (var i in m) {if (m[i] != '') {var o = {};o[m[i]] = 1;r.push(o);}}return r;}" + - "\"}},{\"reduce\":{\"language\":\"javascript\",\"source\":\"" + - "function(v) {var r = {};for (var i in v) {for(var w in v[i]) {if (w in r) r[w] += v[i][w];" + - "else r[w] = v[i][w];}}return [r];}\"}}]}"; - - MapReduceOperation mrOp = - new MapReduceOperation.Builder(BinaryValue.unsafeCreate(query.getBytes())) - .build(); - - cluster.execute(mrOp); - mrOp.await(); - assertTrue(mrOp.isSuccess()); - ArrayNode resultList = mrOp.get().getResults().get(1); - - // The query should return one result which is a JSON array containing a - // single JSON object that is a asSet of word counts. - assertEquals(resultList.size(), 1); - - String json = resultList.get(0).toString(); - ObjectMapper oMapper = new ObjectMapper(); - @SuppressWarnings("unchecked") - Map resultMap = oMapper.readValue(json, Map.class); - - resetAndEmptyBucket(namespace); - - return resultMap; } } diff --git a/src/test/java/com/basho/riak/client/core/operations/itest/ITestSecondaryIndexQueryOp.java b/src/test/java/com/basho/riak/client/core/operations/itest/ITestSecondaryIndexQueryOp.java index 5d7fa708f..bffb71596 100644 --- a/src/test/java/com/basho/riak/client/core/operations/itest/ITestSecondaryIndexQueryOp.java +++ b/src/test/java/com/basho/riak/client/core/operations/itest/ITestSecondaryIndexQueryOp.java @@ -15,6 +15,7 @@ */ package com.basho.riak.client.core.operations.itest; +import com.basho.riak.client.core.StreamingRiakFuture; import com.basho.riak.client.core.operations.SecondaryIndexQueryOperation; import com.basho.riak.client.core.operations.StoreOperation; import com.basho.riak.client.core.query.Location; @@ -24,10 +25,17 @@ import com.basho.riak.client.core.query.indexes.LongIntIndex; import com.basho.riak.client.core.query.indexes.StringBinIndex; import com.basho.riak.client.core.util.BinaryValue; -import org.junit.*; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; +import java.util.LinkedList; +import java.util.List; import java.util.Random; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import static org.junit.Assert.*; @@ -80,6 +88,13 @@ public void testSingleQuerySingleResponseDefaultType() throws InterruptedExcepti testSingleQuerySingleResponse(defaultTypeNamespace); } + @Test + public void testSingleQuerySingleResponseDefaultTypeStreaming() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); + testSingleQuerySingleResponseStreaming(defaultTypeNamespace); + } + @Test public void testSingleQuerySingleResponseTestType() throws InterruptedException, ExecutionException { @@ -133,6 +148,12 @@ public void testNoSortWithNoPagingTestType() throws InterruptedException, Execut testNoSortWithNoPaging(typedNamespace); } + @Test + public void testNoSortWithNoPagingTestTypeStreaming() throws InterruptedException, ExecutionException + { + testNoSortWithNoPagingStreaming(typedNamespace); + } + @Test public void testSortWithNoPagingDefaultType() throws InterruptedException, ExecutionException { @@ -221,6 +242,14 @@ public void testSortWithPagingTestType() throws InterruptedException, ExecutionE testSortWithPaging(typedNamespace); } + @Test + public void testSortWithPagingTestTypeStreaming() throws InterruptedException, ExecutionException + { + Assume.assumeTrue(test2i); + Assume.assumeTrue(testBucketType); + testSortWithPagingStreaming(typedNamespace); + } + @Test public void testRegexTermFilterDefaultType() throws InterruptedException, ExecutionException { @@ -287,6 +316,70 @@ private void testSingleQuerySingleResponse(Namespace namespace) throws Interrupt assertEquals(keyBase + "5", response.getEntryList().get(0).getObjectKey().toString()); } + private void testSingleQuerySingleResponseStreaming(Namespace namespace) throws InterruptedException, ExecutionException + { + { + SecondaryIndexQueryOperation.Query query = + new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) + .withIndexKey(BinaryValue.unsafeCreate(String.valueOf(5L).getBytes())) + .build(); + + SecondaryIndexQueryOperation queryOp = + new SecondaryIndexQueryOperation.Builder(query).streamResults(true).build(); + + final StreamingRiakFuture + future = cluster.execute(queryOp); + + SecondaryIndexQueryOperation.Response syncResponse = queryOp.get(); + + List streamedResponses = new LinkedList<>(); + for (SecondaryIndexQueryOperation.Response r : future.getResultsQueue()) + { + streamedResponses.addAll(r.getEntryList()); + } + + assertEquals(0, syncResponse.getEntryList().size()); + assertFalse(syncResponse.hasContinuation()); + + assertEquals(1, streamedResponses.size()); + final SecondaryIndexQueryOperation.Response.Entry entry = streamedResponses.get(0); + assertFalse(entry.hasIndexKey()); + assertEquals(keyBase + "5", entry.getObjectKey().toString()); + assertFalse(syncResponse.hasContinuation()); + } + + { + SecondaryIndexQueryOperation.Query query2 = + new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) + .withIndexKey(BinaryValue.unsafeCreate(String.valueOf(5L).getBytes())) + .withReturnKeyAndIndex(true) + .build(); + + SecondaryIndexQueryOperation queryOp2 = + new SecondaryIndexQueryOperation.Builder(query2).streamResults(true).build(); + + final StreamingRiakFuture + future2 = cluster.execute(queryOp2); + + SecondaryIndexQueryOperation.Response syncResponse2 = queryOp2.get(); + + LinkedList streamedResponses2 = new LinkedList<>(); + for (SecondaryIndexQueryOperation.Response r : future2.getResultsQueue()) + { + streamedResponses2.addAll(r.getEntryList()); + } + + assertEquals(0, syncResponse2.getEntryList().size()); + assertFalse(syncResponse2.hasContinuation()); + + assertEquals(1, streamedResponses2.size()); + final SecondaryIndexQueryOperation.Response.Entry entry2 = streamedResponses2.get(0); + assertTrue(entry2.hasIndexKey()); + assertEquals(BinaryValue.unsafeCreate("5".getBytes()), entry2.getIndexKey()); + assertEquals(keyBase + "5", entry2.getObjectKey().toString()); + } + } + private void testSingleQueryMultipleResponse(Namespace namespace) throws InterruptedException, ExecutionException { SecondaryIndexQueryOperation.Query query = @@ -296,8 +389,7 @@ private void testSingleQueryMultipleResponse(Namespace namespace) throws Interru .build(); SecondaryIndexQueryOperation queryOp = - new SecondaryIndexQueryOperation.Builder(query) - .build(); + new SecondaryIndexQueryOperation.Builder(query).build(); cluster.execute(queryOp); SecondaryIndexQueryOperation.Response response = queryOp.get(); @@ -306,16 +398,13 @@ private void testSingleQueryMultipleResponse(Namespace namespace) throws Interru assertFalse(response.getEntryList().get(0).hasIndexKey()); assertEquals(keyBase + "0", response.getEntryList().get(0).getObjectKey().toString()); - query = - new SecondaryIndexQueryOperation.Query.Builder(namespace, allFivesIndexName) + query = new SecondaryIndexQueryOperation.Query.Builder(namespace, allFivesIndexName) .withIndexKey(BinaryValue.unsafeCreate(String.valueOf(5L).getBytes())) .withPaginationSort(true) .withReturnKeyAndIndex(true) .build(); - queryOp = - new SecondaryIndexQueryOperation.Builder(query) - .build(); + queryOp = new SecondaryIndexQueryOperation.Builder(query).build(); cluster.execute(queryOp); response = queryOp.get(); @@ -346,17 +435,14 @@ private void testRangeQuery(Namespace namespace) throws InterruptedException, Ex assertFalse(response.getEntryList().get(0).hasIndexKey()); assertEquals(keyBase + "5", response.getEntryList().get(0).getObjectKey().toString()); - query = - new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) + query = new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) .withRangeStart(BinaryValue.unsafeCreate(String.valueOf(5L).getBytes())) .withRangeEnd(BinaryValue.unsafeCreate(String.valueOf(20L).getBytes())) .withReturnKeyAndIndex(true) .withPaginationSort(true) .build(); - queryOp = - new SecondaryIndexQueryOperation.Builder(query) - .build(); + queryOp = new SecondaryIndexQueryOperation.Builder(query).build(); cluster.execute(queryOp); response = queryOp.get(); @@ -386,6 +472,33 @@ private void testNoSortWithNoPaging(Namespace namespace) throws InterruptedExcep assertEquals(100, response.getEntryList().size()); } + private void testNoSortWithNoPagingStreaming(Namespace namespace) throws InterruptedException, ExecutionException + { + SecondaryIndexQueryOperation.Query query = + new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) + .withRangeStart(BinaryValue.unsafeCreate(String.valueOf(0L).getBytes())) + .withRangeEnd(BinaryValue.unsafeCreate(String.valueOf(100L).getBytes())) + .withPaginationSort(false) + .build(); + + SecondaryIndexQueryOperation queryOp = + new SecondaryIndexQueryOperation.Builder(query) + .streamResults(true) + .build(); + + final StreamingRiakFuture + future = cluster.execute(queryOp); + + StreamingResultsGatherer streamingResultsGatherer = new StreamingResultsGatherer(future).invoke(); + List resultsList = streamingResultsGatherer.getResultsList(); + BinaryValue continuation = streamingResultsGatherer.getContinuation(); + + assertEquals(100, resultsList.size()); + assertEquals(0, future.get().getEntryList().size()); + assertNull(continuation); + assertFalse(future.get().hasContinuation()); + } + private void testSortWithNoPaging(Namespace namespace) throws InterruptedException, ExecutionException { SecondaryIndexQueryOperation.Query query = @@ -404,7 +517,7 @@ private void testSortWithNoPaging(Namespace namespace) throws InterruptedExcepti assertEquals(100, response.getEntryList().size()); - AssertLongObjectsInOrder(response); + AssertLongObjectsInOrder(response.getEntryList()); } private void testNoSortWithPaging(Namespace namespace) throws InterruptedException, ExecutionException @@ -448,7 +561,38 @@ private void testSortWithPaging(Namespace namespace) throws InterruptedException assertEquals(20, response.getEntryList().size()); - AssertLongObjectsInOrder(response); + AssertLongObjectsInOrder(response.getEntryList()); + } + + private void testSortWithPagingStreaming(Namespace namespace) throws InterruptedException, ExecutionException + { + SecondaryIndexQueryOperation.Query query = + new SecondaryIndexQueryOperation.Query.Builder(namespace, incrementingIndexName) + .withRangeStart(BinaryValue.unsafeCreate(String.valueOf(0L).getBytes())) + .withRangeEnd(BinaryValue.unsafeCreate(String.valueOf(100L).getBytes())) + .withPaginationSort(true) + .withMaxResults(20) + .build(); + + SecondaryIndexQueryOperation queryOp = + new SecondaryIndexQueryOperation.Builder(query) + .streamResults(true) + .build(); + + final StreamingRiakFuture + future = cluster.execute(queryOp); + + StreamingResultsGatherer streamingResultsGatherer = new StreamingResultsGatherer(future).invoke(); + List resultsList = streamingResultsGatherer.getResultsList(); + BinaryValue continuation = streamingResultsGatherer.getContinuation(); + + assertEquals(20, resultsList.size()); + assertEquals(0, future.get().getEntryList().size()); + assertTrue(future.get().hasContinuation()); + assertNotNull(continuation); + assertEquals(continuation, future.get().getContinuation()); + + AssertLongObjectsInOrder(resultsList); } private void testRegexTermFilter(Namespace namespace) throws InterruptedException, ExecutionException @@ -463,8 +607,7 @@ private void testRegexTermFilter(Namespace namespace) throws InterruptedExceptio .build(); SecondaryIndexQueryOperation queryOp = - new SecondaryIndexQueryOperation.Builder(query) - .build(); + new SecondaryIndexQueryOperation.Builder(query).build(); cluster.execute(queryOp); SecondaryIndexQueryOperation.Response response = queryOp.get(); @@ -521,17 +664,89 @@ private static void setupIndexTestData(Namespace ns) } } - private void AssertLongObjectsInOrder(SecondaryIndexQueryOperation.Response response) + private void AssertLongObjectsInOrder(List entryList) { - final String firstKey = response.getEntryList().get(0).getObjectKey().toString(); + final String firstKey = entryList.get(0).getObjectKey().toString(); Long previousKey = Long.parseLong(firstKey.substring(keyBase.length())); - for (int j = 1; j < response.getEntryList().size(); j++) + for (int j = 1; j < entryList.size(); j++) { - String fullKey = response.getEntryList().get(j).getObjectKey().toString(); + String fullKey = entryList.get(j).getObjectKey().toString(); Long currentKey = Long.parseLong(fullKey.substring(keyBase.length())); assertTrue(previousKey <= currentKey); previousKey = currentKey; } } + + private class StreamingResultsGatherer + { + private StreamingRiakFuture future; + private List resultsList; + private BinaryValue continuation; + + public StreamingResultsGatherer(StreamingRiakFuture future) + { + this.future = future; + } + + public List getResultsList() + { + return resultsList; + } + + public BinaryValue getContinuation() + { + return continuation; + } + + public StreamingResultsGatherer invoke() throws InterruptedException + { + final BlockingQueue resultsQueue = future.getResultsQueue(); + + resultsList = new LinkedList<>(); + + continuation = null; + int timeouts = 0; + + while (!future.isDone()) + { + final SecondaryIndexQueryOperation.Response response = resultsQueue.poll(5, TimeUnit.MILLISECONDS); + + if (response == null) + { + if (++timeouts == 10) + { + break; + } + else + { + continue; + } + } + + continuation = addResponseToResults(resultsList, continuation, response); + } + + for (SecondaryIndexQueryOperation.Response response : resultsQueue) + { + continuation = addResponseToResults(resultsList, continuation, response); + } + return this; + } + + private BinaryValue addResponseToResults(List resultsList, + BinaryValue continuation, + SecondaryIndexQueryOperation.Response response) + { + resultsList.addAll(response.getEntryList()); + + if (response.hasContinuation() && continuation == null) + { + continuation = response.getContinuation(); + } + + return continuation; + } + } }